!htc #19514,#18958 make config in HTTP non-case class + from reference

This commit is contained in:
Konrad Malawski 2016-01-20 16:11:09 +01:00 committed by Johan Andrén
parent 97df7bf689
commit 9b4b68f0c9
17 changed files with 215 additions and 124 deletions

View file

@ -79,6 +79,17 @@ In Akka 2.4.x this is formulated like so:
.. includecode:: ../code/docs/stream/MigrationsScala.scala#expand-state .. includecode:: ../code/docs/stream/MigrationsScala.scala#expand-state
Changes in Akka HTTP
====================
Routing settings parameter name
-------------------------------
``RoutingSettings`` were previously the only setting available on ``RequestContext``,
and were accessible via ``settings``. We now made it possible to configure the parsers
settings as well, so ``RoutingSettings`` is now ``routingSettings`` and ``ParserSetttings`` is
now accessible via ``parserSettings``.
Changed Sources / Sinks Changed Sources / Sinks
======================= =======================

View file

@ -19,14 +19,14 @@ import akka.http.impl.util._
import akka.http.scaladsl.model.headers.`User-Agent` import akka.http.scaladsl.model.headers.`User-Agent`
final case class ClientConnectionSettings( final class ClientConnectionSettings(
userAgentHeader: Option[`User-Agent`], val userAgentHeader: Option[`User-Agent`],
connectingTimeout: FiniteDuration, val connectingTimeout: FiniteDuration,
idleTimeout: Duration, val idleTimeout: Duration,
requestHeaderSizeHint: Int, val requestHeaderSizeHint: Int,
websocketRandomFactory: () Random, val websocketRandomFactory: () Random,
socketOptions: immutable.Traversable[SocketOption], val socketOptions: immutable.Traversable[SocketOption],
parserSettings: ParserSettings) { val parserSettings: ParserSettings) {
require(connectingTimeout >= Duration.Zero, "connectingTimeout must be >= 0") require(connectingTimeout >= Duration.Zero, "connectingTimeout must be >= 0")
require(requestHeaderSizeHint > 0, "request-size-hint must be > 0") require(requestHeaderSizeHint > 0, "request-size-hint must be > 0")
@ -35,14 +35,14 @@ final case class ClientConnectionSettings(
object ClientConnectionSettings extends SettingsCompanion[ClientConnectionSettings]("akka.http.client") { object ClientConnectionSettings extends SettingsCompanion[ClientConnectionSettings]("akka.http.client") {
def fromSubConfig(root: Config, inner: Config) = { def fromSubConfig(root: Config, inner: Config) = {
val c = inner.withFallback(root.getConfig(prefix)) val c = inner.withFallback(root.getConfig(prefix))
apply( new ClientConnectionSettings(
c.getString("user-agent-header").toOption.map(`User-Agent`(_)), userAgentHeader = c.getString("user-agent-header").toOption.map(`User-Agent`(_)),
c getFiniteDuration "connecting-timeout", connectingTimeout = c getFiniteDuration "connecting-timeout",
c getPotentiallyInfiniteDuration "idle-timeout", idleTimeout = c getPotentiallyInfiniteDuration "idle-timeout",
c getIntBytes "request-header-size-hint", requestHeaderSizeHint = c getIntBytes "request-header-size-hint",
Randoms.SecureRandomInstances, // can currently only be overridden from code websocketRandomFactory = Randoms.SecureRandomInstances, // can currently only be overridden from code
SocketOptionSettings.fromSubConfig(root, c.getConfig("socket-options")), socketOptions = SocketOptionSettings.fromSubConfig(root, c.getConfig("socket-options")),
ParserSettings.fromSubConfig(root, c.getConfig("parsing"))) parserSettings = ParserSettings.fromSubConfig(root, c.getConfig("parsing")))
} }
/** /**

View file

@ -27,13 +27,13 @@ object ConnectionPoolSetup {
ConnectionPoolSetup(settings, connectionContext, log) ConnectionPoolSetup(settings, connectionContext, log)
} }
final case class ConnectionPoolSettings( final class ConnectionPoolSettings(
maxConnections: Int, val maxConnections: Int,
maxRetries: Int, val maxRetries: Int,
maxOpenRequests: Int, val maxOpenRequests: Int,
pipeliningLimit: Int, val pipeliningLimit: Int,
idleTimeout: Duration, val idleTimeout: Duration,
connectionSettings: ClientConnectionSettings) { val connectionSettings: ClientConnectionSettings) {
require(maxConnections > 0, "max-connections must be > 0") require(maxConnections > 0, "max-connections must be > 0")
require(maxRetries >= 0, "max-retries must be >= 0") require(maxRetries >= 0, "max-retries must be >= 0")
@ -44,7 +44,7 @@ final case class ConnectionPoolSettings(
object ConnectionPoolSettings extends SettingsCompanion[ConnectionPoolSettings]("akka.http.host-connection-pool") { object ConnectionPoolSettings extends SettingsCompanion[ConnectionPoolSettings]("akka.http.host-connection-pool") {
def fromSubConfig(root: Config, c: Config) = { def fromSubConfig(root: Config, c: Config) = {
apply( new ConnectionPoolSettings(
c getInt "max-connections", c getInt "max-connections",
c getInt "max-retries", c getInt "max-retries",
c getInt "max-open-requests", c getInt "max-open-requests",

View file

@ -4,32 +4,33 @@
package akka.http package akka.http
import java.util.Locale import akka.actor.{ ActorRefFactory, ActorSystem }
import akka.actor.ActorSystem import akka.stream.{ ActorMaterializer, Materializer }
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import akka.http.scaladsl.model.{ StatusCode, HttpMethod, Uri } import akka.http.scaladsl.model.{ StatusCode, HttpMethod, Uri }
import akka.http.impl.util._ import akka.http.impl.util._
import akka.http.impl.engine.parsing.HttpHeaderParser import akka.http.impl.engine.parsing.{ BodyPartParser, HttpHeaderParser }
final case class ParserSettings( final class ParserSettings(
maxUriLength: Int, val maxUriLength: Int,
maxMethodLength: Int, val maxMethodLength: Int,
maxResponseReasonLength: Int, val maxResponseReasonLength: Int,
maxHeaderNameLength: Int, val maxHeaderNameLength: Int,
maxHeaderValueLength: Int, val maxHeaderValueLength: Int,
maxHeaderCount: Int, val maxHeaderCount: Int,
maxContentLength: Long, val maxContentLength: Long,
maxChunkExtLength: Int, val maxChunkExtLength: Int,
maxChunkSize: Int, val maxChunkSize: Int,
uriParsingMode: Uri.ParsingMode, val uriParsingMode: Uri.ParsingMode,
cookieParsingMode: ParserSettings.CookieParsingMode, val cookieParsingMode: ParserSettings.CookieParsingMode,
illegalHeaderWarnings: Boolean, val illegalHeaderWarnings: Boolean,
errorLoggingVerbosity: ParserSettings.ErrorLoggingVerbosity, val errorLoggingVerbosity: ParserSettings.ErrorLoggingVerbosity,
headerValueCacheLimits: Map[String, Int], val headerValueCacheLimits: Map[String, Int],
includeTlsSessionInfoHeader: Boolean, val includeTlsSessionInfoHeader: Boolean,
customMethods: String Option[HttpMethod], val customMethods: String Option[HttpMethod],
customStatusCodes: Int Option[StatusCode]) extends HttpHeaderParser.Settings { val customStatusCodes: Int Option[StatusCode])
extends BodyPartParser.Settings {
require(maxUriLength > 0, "max-uri-length must be > 0") require(maxUriLength > 0, "max-uri-length must be > 0")
require(maxMethodLength > 0, "max-method-length must be > 0") require(maxMethodLength > 0, "max-method-length must be > 0")
@ -41,9 +42,9 @@ final case class ParserSettings(
require(maxChunkExtLength > 0, "max-chunk-ext-length must be > 0") require(maxChunkExtLength > 0, "max-chunk-ext-length must be > 0")
require(maxChunkSize > 0, "max-chunk-size must be > 0") require(maxChunkSize > 0, "max-chunk-size must be > 0")
val defaultHeaderValueCacheLimit: Int = headerValueCacheLimits("default") override val defaultHeaderValueCacheLimit: Int = headerValueCacheLimits("default")
def headerValueCacheLimit(headerName: String): Int = override def headerValueCacheLimit(headerName: String): Int =
headerValueCacheLimits.getOrElse(headerName, defaultHeaderValueCacheLimit) headerValueCacheLimits.getOrElse(headerName, defaultHeaderValueCacheLimit)
def withCustomMethods(methods: HttpMethod*): ParserSettings = { def withCustomMethods(methods: HttpMethod*): ParserSettings = {
@ -54,6 +55,41 @@ final case class ParserSettings(
val map = codes.map(c c.intValue -> c).toMap val map = codes.map(c c.intValue -> c).toMap
copy(customStatusCodes = map.get) copy(customStatusCodes = map.get)
} }
def copy(maxUriLength: Int = maxUriLength,
maxMethodLength: Int = maxMethodLength,
maxResponseReasonLength: Int = maxResponseReasonLength,
maxHeaderNameLength: Int = maxHeaderNameLength,
maxHeaderValueLength: Int = maxHeaderValueLength,
maxHeaderCount: Int = maxHeaderCount,
maxContentLength: Long = maxContentLength,
maxChunkExtLength: Int = maxChunkExtLength,
maxChunkSize: Int = maxChunkSize,
uriParsingMode: Uri.ParsingMode = uriParsingMode,
cookieParsingMode: ParserSettings.CookieParsingMode = cookieParsingMode,
illegalHeaderWarnings: Boolean = illegalHeaderWarnings,
errorLoggingVerbosity: ParserSettings.ErrorLoggingVerbosity = errorLoggingVerbosity,
headerValueCacheLimits: Map[String, Int] = headerValueCacheLimits,
includeTlsSessionInfoHeader: Boolean = includeTlsSessionInfoHeader,
customMethods: String Option[HttpMethod] = customMethods,
customStatusCodes: Int Option[StatusCode] = customStatusCodes): ParserSettings =
new ParserSettings(maxUriLength,
maxMethodLength,
maxResponseReasonLength,
maxHeaderNameLength,
maxHeaderValueLength,
maxHeaderCount,
maxContentLength,
maxChunkExtLength,
maxChunkSize,
uriParsingMode,
cookieParsingMode,
illegalHeaderWarnings,
errorLoggingVerbosity,
headerValueCacheLimits,
includeTlsSessionInfoHeader,
customMethods,
customStatusCodes)
} }
object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsing") { object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsing") {
@ -61,7 +97,7 @@ object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsi
val c = inner.withFallback(root.getConfig(prefix)) val c = inner.withFallback(root.getConfig(prefix))
val cacheConfig = c getConfig "header-cache" val cacheConfig = c getConfig "header-cache"
apply( new ParserSettings(
c getIntBytes "max-uri-length", c getIntBytes "max-uri-length",
c getIntBytes "max-method-length", c getIntBytes "max-method-length",
c getIntBytes "max-response-reason-length", c getIntBytes "max-response-reason-length",
@ -130,5 +166,8 @@ object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsi
* Java API * Java API
*/ */
def create(configOverrides: String): ParserSettings = ParserSettings(configOverrides) def create(configOverrides: String): ParserSettings = ParserSettings(configOverrides)
implicit def default(implicit refFactory: ActorRefFactory): ParserSettings =
apply(actorSystem)
} }

View file

@ -22,21 +22,21 @@ import akka.http.impl.util._
import akka.http.scaladsl.model.HttpHeader import akka.http.scaladsl.model.HttpHeader
import akka.http.scaladsl.model.headers.{ Host, Server } import akka.http.scaladsl.model.headers.{ Host, Server }
final case class ServerSettings( final class ServerSettings(
serverHeader: Option[Server], val serverHeader: Option[Server],
timeouts: ServerSettings.Timeouts, val timeouts: ServerSettings.Timeouts,
maxConnections: Int, val maxConnections: Int,
pipeliningLimit: Int, val pipeliningLimit: Int,
remoteAddressHeader: Boolean, val remoteAddressHeader: Boolean,
rawRequestUriHeader: Boolean, val rawRequestUriHeader: Boolean,
transparentHeadRequests: Boolean, val transparentHeadRequests: Boolean,
verboseErrorMessages: Boolean, val verboseErrorMessages: Boolean,
responseHeaderSizeHint: Int, val responseHeaderSizeHint: Int,
backlog: Int, val backlog: Int,
socketOptions: immutable.Traversable[SocketOption], val socketOptions: immutable.Traversable[SocketOption],
defaultHostHeader: Host, val defaultHostHeader: Host,
websocketRandomFactory: () Random, val websocketRandomFactory: () Random,
parserSettings: ParserSettings) { val parserSettings: ParserSettings) {
require(0 < maxConnections, "max-connections must be > 0") require(0 < maxConnections, "max-connections must be > 0")
require(0 < pipeliningLimit && pipeliningLimit <= 1024, "pipelining-limit must be > 0 and <= 1024") require(0 < pipeliningLimit && pipeliningLimit <= 1024, "pipelining-limit must be > 0 and <= 1024")
@ -45,18 +45,19 @@ final case class ServerSettings(
} }
object ServerSettings extends SettingsCompanion[ServerSettings]("akka.http.server") { object ServerSettings extends SettingsCompanion[ServerSettings]("akka.http.server") {
final case class Timeouts(idleTimeout: Duration, final class Timeouts(
requestTimeout: Duration, val idleTimeout: Duration,
bindTimeout: FiniteDuration) { val requestTimeout: Duration,
val bindTimeout: FiniteDuration) {
require(idleTimeout > Duration.Zero, "idleTimeout must be infinite or > 0") require(idleTimeout > Duration.Zero, "idleTimeout must be infinite or > 0")
require(requestTimeout > Duration.Zero, "requestTimeout must be infinite or > 0") require(requestTimeout > Duration.Zero, "requestTimeout must be infinite or > 0")
require(bindTimeout > Duration.Zero, "bindTimeout must be > 0") require(bindTimeout > Duration.Zero, "bindTimeout must be > 0")
} }
implicit def timeoutsShortcut(s: ServerSettings): Timeouts = s.timeouts implicit def timeoutsShortcut(s: ServerSettings): Timeouts = s.timeouts
def fromSubConfig(root: Config, c: Config) = apply( def fromSubConfig(root: Config, c: Config) = new ServerSettings(
c.getString("server-header").toOption.map(Server(_)), c.getString("server-header").toOption.map(Server(_)),
Timeouts( new Timeouts(
c getPotentiallyInfiniteDuration "idle-timeout", c getPotentiallyInfiniteDuration "idle-timeout",
c getPotentiallyInfiniteDuration "request-timeout", c getPotentiallyInfiniteDuration "request-timeout",
c getFiniteDuration "bind-timeout"), c getFiniteDuration "bind-timeout"),

View file

@ -5,8 +5,6 @@
package akka.http.impl.engine.parsing package akka.http.impl.engine.parsing
import akka.NotUsed import akka.NotUsed
import akka.http.ParserSettings
import akka.stream.impl.fusing.GraphInterpreter
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.parboiled2.CharPredicate import akka.parboiled2.CharPredicate
@ -27,7 +25,7 @@ import akka.stream.impl.fusing.SubSource
private[http] final class BodyPartParser(defaultContentType: ContentType, private[http] final class BodyPartParser(defaultContentType: ContentType,
boundary: String, boundary: String,
log: LoggingAdapter, log: LoggingAdapter,
settings: BodyPartParser.Settings = BodyPartParser.defaultSettings) settings: BodyPartParser.Settings)
extends PushPullStage[ByteString, BodyPartParser.Output] { extends PushPullStage[ByteString, BodyPartParser.Output] {
import BodyPartParser._ import BodyPartParser._
import settings._ import settings._
@ -277,28 +275,9 @@ private[http] object BodyPartParser {
final case class EntityPart(data: ByteString) extends Output final case class EntityPart(data: ByteString) extends Output
final case class ParseError(info: ErrorInfo) extends PartStart final case class ParseError(info: ErrorInfo) extends PartStart
final case class Settings( abstract class Settings extends HttpHeaderParser.Settings {
maxHeaderNameLength: Int, def maxHeaderCount: Int
maxHeaderValueLength: Int, def illegalHeaderWarnings: Boolean
maxHeaderCount: Int, def defaultHeaderValueCacheLimit: Int
illegalHeaderWarnings: Boolean,
headerValueCacheLimit: Int,
uriParsingMode: Uri.ParsingMode,
cookieParsingMode: ParserSettings.CookieParsingMode) extends HttpHeaderParser.Settings {
require(maxHeaderNameLength > 0, "maxHeaderNameLength must be > 0")
require(maxHeaderValueLength > 0, "maxHeaderValueLength must be > 0")
require(maxHeaderCount > 0, "maxHeaderCount must be > 0")
require(headerValueCacheLimit >= 0, "headerValueCacheLimit must be >= 0")
def headerValueCacheLimit(headerName: String) = headerValueCacheLimit
} }
// TODO: load from config
val defaultSettings = Settings(
maxHeaderNameLength = 64,
maxHeaderValueLength = 8192,
maxHeaderCount = 64,
illegalHeaderWarnings = true,
headerValueCacheLimit = 8,
uriParsingMode = Uri.ParsingMode.Relaxed,
cookieParsingMode = ParserSettings.CookieParsingMode.RFC6265)
} }

View file

@ -408,7 +408,7 @@ private[engine] final class HttpHeaderParser private (
private[http] object HttpHeaderParser { private[http] object HttpHeaderParser {
import SpecializedHeaderValueParsers._ import SpecializedHeaderValueParsers._
trait Settings extends HeaderParser.Settings { abstract class Settings extends HeaderParser.Settings {
def maxHeaderNameLength: Int def maxHeaderNameLength: Int
def maxHeaderValueLength: Int def maxHeaderValueLength: Int
def headerValueCacheLimit(headerName: String): Int def headerValueCacheLimit(headerName: String): Int

View file

@ -156,7 +156,7 @@ private[http] object HeaderParser {
"www-authenticate", "www-authenticate",
"x-forwarded-for") "x-forwarded-for")
trait Settings { abstract class Settings {
def uriParsingMode: Uri.ParsingMode def uriParsingMode: Uri.ParsingMode
def cookieParsingMode: ParserSettings.CookieParsingMode def cookieParsingMode: ParserSettings.CookieParsingMode
} }

View file

@ -5,6 +5,7 @@
package akka.http.scaladsl.server package akka.http.scaladsl.server
import scala.concurrent.{ Future, ExecutionContextExecutor } import scala.concurrent.{ Future, ExecutionContextExecutor }
import akka.http.ParserSettings
import akka.stream.Materializer import akka.stream.Materializer
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.http.scaladsl.marshalling.ToResponseMarshallable import akka.http.scaladsl.marshalling.ToResponseMarshallable
@ -42,6 +43,11 @@ trait RequestContext {
*/ */
def settings: RoutingSettings def settings: RoutingSettings
/**
* The default ParserSettings to be used for configuring directives.
*/
def parserSettings: ParserSettings
/** /**
* Returns a copy of this context with the given fields updated. * Returns a copy of this context with the given fields updated.
*/ */
@ -91,7 +97,12 @@ trait RequestContext {
/** /**
* Returns a copy of this context with the new RoutingSettings. * Returns a copy of this context with the new RoutingSettings.
*/ */
def withSettings(settings: RoutingSettings): RequestContext def withRoutingSettings(settings: RoutingSettings): RequestContext
/**
* Returns a copy of this context with the new [[ParserSettings]].
*/
def withParserSettings(settings: ParserSettings): RequestContext
/** /**
* Returns a copy of this context with the HttpRequest transformed by the given function. * Returns a copy of this context with the HttpRequest transformed by the given function.

View file

@ -5,8 +5,9 @@
package akka.http.scaladsl.server package akka.http.scaladsl.server
import scala.concurrent.{ Future, ExecutionContextExecutor } import scala.concurrent.{ Future, ExecutionContextExecutor }
import akka.stream.Materializer import akka.stream.{ ActorMaterializer, Materializer }
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.http.ParserSettings
import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable } import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable }
import akka.http.scaladsl.model._ import akka.http.scaladsl.model._
import akka.http.scaladsl.util.FastFuture import akka.http.scaladsl.util.FastFuture
@ -21,13 +22,17 @@ private[http] class RequestContextImpl(
val executionContext: ExecutionContextExecutor, val executionContext: ExecutionContextExecutor,
val materializer: Materializer, val materializer: Materializer,
val log: LoggingAdapter, val log: LoggingAdapter,
val settings: RoutingSettings) extends RequestContext { val settings: RoutingSettings,
val parserSettings: ParserSettings) extends RequestContext {
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings, parserSettings: ParserSettings)(implicit ec: ExecutionContext, materializer: Materializer) =
this(request, request.uri.path, ec, materializer, log, settings, parserSettings)
def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContextExecutor, materializer: Materializer) = def this(request: HttpRequest, log: LoggingAdapter, settings: RoutingSettings)(implicit ec: ExecutionContextExecutor, materializer: Materializer) =
this(request, request.uri.path, ec, materializer, log, settings) this(request, request.uri.path, ec, materializer, log, settings, ParserSettings(ActorMaterializer.downcast(materializer).system))
def reconfigure(executionContext: ExecutionContextExecutor, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext = def reconfigure(executionContext: ExecutionContextExecutor, materializer: Materializer, log: LoggingAdapter, settings: RoutingSettings): RequestContext =
copy(executionContext = executionContext, materializer = materializer, log = log, settings = settings) copy(executionContext = executionContext, materializer = materializer, log = log, routingSettings = settings)
override def complete(trm: ToResponseMarshallable): Future[RouteResult] = override def complete(trm: ToResponseMarshallable): Future[RouteResult] =
trm(request)(executionContext) trm(request)(executionContext)
@ -56,8 +61,11 @@ private[http] class RequestContextImpl(
override def withLog(log: LoggingAdapter): RequestContext = override def withLog(log: LoggingAdapter): RequestContext =
if (log != this.log) copy(log = log) else this if (log != this.log) copy(log = log) else this
override def withSettings(settings: RoutingSettings): RequestContext = override def withRoutingSettings(routingSettings: RoutingSettings): RequestContext =
if (settings != this.settings) copy(settings = settings) else this if (routingSettings != this.settings) copy(routingSettings = routingSettings) else this
override def withParserSettings(parserSettings: ParserSettings): RequestContext =
if (parserSettings != this.parserSettings) copy(parserSettings = parserSettings) else this
override def mapRequest(f: HttpRequest HttpRequest): RequestContext = override def mapRequest(f: HttpRequest HttpRequest): RequestContext =
copy(request = f(request)) copy(request = f(request))
@ -86,6 +94,7 @@ private[http] class RequestContextImpl(
executionContext: ExecutionContextExecutor = executionContext, executionContext: ExecutionContextExecutor = executionContext,
materializer: Materializer = materializer, materializer: Materializer = materializer,
log: LoggingAdapter = log, log: LoggingAdapter = log,
settings: RoutingSettings = settings) = routingSettings: RoutingSettings = settings,
new RequestContextImpl(request, unmatchedPath, executionContext, materializer, log, settings) parserSettings: ParserSettings = parserSettings) =
new RequestContextImpl(request, unmatchedPath, executionContext, materializer, log, routingSettings, parserSettings)
} }

View file

@ -5,7 +5,9 @@
package akka.http.scaladsl.server package akka.http.scaladsl.server
import akka.NotUsed import akka.NotUsed
import akka.stream.Materializer import akka.actor.ActorSystem
import akka.http.ParserSettings
import akka.stream.{ ActorMaterializer, Materializer }
import scala.concurrent.{ ExecutionContextExecutor, Future } import scala.concurrent.{ ExecutionContextExecutor, Future }
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
@ -23,6 +25,7 @@ object Route {
* "Seals" a route by wrapping it with exception handling and rejection conversion. * "Seals" a route by wrapping it with exception handling and rejection conversion.
*/ */
def seal(route: Route)(implicit routingSettings: RoutingSettings, def seal(route: Route)(implicit routingSettings: RoutingSettings,
parserSettings: ParserSettings = null,
rejectionHandler: RejectionHandler = RejectionHandler.default, rejectionHandler: RejectionHandler = RejectionHandler.default,
exceptionHandler: ExceptionHandler = null): Route = { exceptionHandler: ExceptionHandler = null): Route = {
import directives.ExecutionDirectives._ import directives.ExecutionDirectives._
@ -39,6 +42,7 @@ object Route {
* This conversion is also implicitly available through [[RouteResult.route2HandlerFlow]]. * This conversion is also implicitly available through [[RouteResult.route2HandlerFlow]].
*/ */
def handlerFlow(route: Route)(implicit routingSettings: RoutingSettings, def handlerFlow(route: Route)(implicit routingSettings: RoutingSettings,
parserSettings: ParserSettings,
materializer: Materializer, materializer: Materializer,
routingLog: RoutingLog, routingLog: RoutingLog,
executionContext: ExecutionContextExecutor = null, executionContext: ExecutionContextExecutor = null,
@ -50,6 +54,7 @@ object Route {
* Turns a `Route` into an async handler function. * Turns a `Route` into an async handler function.
*/ */
def asyncHandler(route: Route)(implicit routingSettings: RoutingSettings, def asyncHandler(route: Route)(implicit routingSettings: RoutingSettings,
parserSettings: ParserSettings,
materializer: Materializer, materializer: Materializer,
routingLog: RoutingLog, routingLog: RoutingLog,
executionContext: ExecutionContextExecutor = null, executionContext: ExecutionContextExecutor = null,
@ -59,10 +64,11 @@ object Route {
{ {
implicit val executionContext = effectiveEC // overrides parameter implicit val executionContext = effectiveEC // overrides parameter
val effectiveParserSettings = if (parserSettings ne null) parserSettings else ParserSettings(ActorMaterializer.downcast(materializer).system)
val sealedRoute = seal(route) val sealedRoute = seal(route)
request request
sealedRoute(new RequestContextImpl(request, routingLog.requestLog(request), routingSettings)).fast sealedRoute(new RequestContextImpl(request, routingLog.requestLog(request), routingSettings, effectiveParserSettings)).fast
.map { .map {
case RouteResult.Complete(response) response case RouteResult.Complete(response) response
case RouteResult.Rejected(rejected) throw new IllegalStateException(s"Unhandled rejections '$rejected', unsealed RejectionHandler?!") case RouteResult.Rejected(rejected) throw new IllegalStateException(s"Unhandled rejections '$rejected', unsealed RejectionHandler?!")

View file

@ -5,6 +5,7 @@
package akka.http.scaladsl.server package akka.http.scaladsl.server
import akka.NotUsed import akka.NotUsed
import akka.http.ParserSettings
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
@ -25,6 +26,7 @@ object RouteResult {
final case class Rejected(rejections: immutable.Seq[Rejection]) extends RouteResult final case class Rejected(rejections: immutable.Seq[Rejection]) extends RouteResult
implicit def route2HandlerFlow(route: Route)(implicit routingSettings: RoutingSettings, implicit def route2HandlerFlow(route: Route)(implicit routingSettings: RoutingSettings,
parserSettings: ParserSettings,
materializer: Materializer, materializer: Materializer,
routingLog: RoutingLog, routingLog: RoutingLog,
executionContext: ExecutionContext = null, executionContext: ExecutionContext = null,

View file

@ -8,17 +8,35 @@ import com.typesafe.config.Config
import akka.actor.ActorRefFactory import akka.actor.ActorRefFactory
import akka.http.impl.util._ import akka.http.impl.util._
case class RoutingSettings( final class RoutingSettings(
verboseErrorMessages: Boolean, val verboseErrorMessages: Boolean,
fileGetConditional: Boolean, val fileGetConditional: Boolean,
renderVanityFooter: Boolean, val renderVanityFooter: Boolean,
rangeCountLimit: Int, val rangeCountLimit: Int,
rangeCoalescingThreshold: Long, val rangeCoalescingThreshold: Long,
decodeMaxBytesPerChunk: Int, val decodeMaxBytesPerChunk: Int,
fileIODispatcher: String) val fileIODispatcher: String) {
def copy(
verboseErrorMessages: Boolean = verboseErrorMessages,
fileGetConditional: Boolean = fileGetConditional,
renderVanityFooter: Boolean = renderVanityFooter,
rangeCountLimit: Int = rangeCountLimit,
rangeCoalescingThreshold: Long = rangeCoalescingThreshold,
decodeMaxBytesPerChunk: Int = decodeMaxBytesPerChunk,
fileIODispatcher: String = fileIODispatcher): RoutingSettings =
new RoutingSettings(
verboseErrorMessages,
fileGetConditional,
renderVanityFooter,
rangeCountLimit,
rangeCoalescingThreshold,
decodeMaxBytesPerChunk,
fileIODispatcher)
}
object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") { object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") {
def fromSubConfig(root: Config, c: Config) = apply( def fromSubConfig(root: Config, c: Config) = new RoutingSettings(
c getBoolean "verbose-error-messages", c getBoolean "verbose-error-messages",
c getBoolean "file-get-conditional", c getBoolean "file-get-conditional",
c getBoolean "render-vanity-footer", c getBoolean "render-vanity-footer",

View file

@ -9,6 +9,7 @@ import scala.concurrent.{ Future, ExecutionContextExecutor }
import scala.collection.immutable import scala.collection.immutable
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.stream.Materializer import akka.stream.Materializer
import akka.http.ParserSettings
import akka.http.scaladsl.server.util.Tuple import akka.http.scaladsl.server.util.Tuple
import akka.http.scaladsl.util.FastFuture import akka.http.scaladsl.util.FastFuture
import akka.http.scaladsl.model._ import akka.http.scaladsl.model._
@ -167,13 +168,13 @@ trait BasicDirectives {
* Runs its inner route with the given alternative [[RoutingSettings]]. * Runs its inner route with the given alternative [[RoutingSettings]].
*/ */
def withSettings(settings: RoutingSettings): Directive0 = def withSettings(settings: RoutingSettings): Directive0 =
mapRequestContext(_ withSettings settings) mapRequestContext(_ withRoutingSettings settings)
/** /**
* Runs the inner route with settings mapped by the given function. * Runs the inner route with settings mapped by the given function.
*/ */
def mapSettings(f: RoutingSettings RoutingSettings): Directive0 = def mapSettings(f: RoutingSettings RoutingSettings): Directive0 =
mapRequestContext(ctx ctx.withSettings(f(ctx.settings))) mapRequestContext(ctx ctx.withRoutingSettings(f(ctx.settings)))
/** /**
* Extracts the [[RoutingSettings]] from the [[RequestContext]]. * Extracts the [[RoutingSettings]] from the [[RequestContext]].
@ -181,6 +182,12 @@ trait BasicDirectives {
def extractSettings: Directive1[RoutingSettings] = def extractSettings: Directive1[RoutingSettings] =
BasicDirectives._extractSettings BasicDirectives._extractSettings
/**
* Extracts the [[akka.http.ParserSettings]] from the [[RequestContext]].
*/
def extractParserSettings: Directive1[ParserSettings] =
BasicDirectives._extractParserSettings
/** /**
* Extracts the [[RequestContext]] itself. * Extracts the [[RequestContext]] itself.
*/ */
@ -195,5 +202,6 @@ object BasicDirectives extends BasicDirectives {
private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer) private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer)
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log) private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings) private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
private val _extractParserSettings: Directive1[ParserSettings] = extract(_.parserSettings)
private val _extractRequestContext: Directive1[RequestContext] = extract(conforms) private val _extractRequestContext: Directive1[RequestContext] = extract(conforms)
} }

View file

@ -76,7 +76,6 @@ trait FileUploadDirectives {
case Some(tuple) provide(tuple) case Some(tuple) provide(tuple)
case None reject(MissingFormFieldRejection(fieldName)) case None reject(MissingFormFieldRejection(fieldName))
} }
} }
object FileUploadDirectives extends FileUploadDirectives object FileUploadDirectives extends FileUploadDirectives

View file

@ -5,10 +5,15 @@
package akka.http.scaladsl.server package akka.http.scaladsl.server
package directives package directives
import akka.actor.ActorSystem
import akka.http.ParserSettings
import akka.http.scaladsl.model.Multipart
import akka.http.scaladsl.model.Multipart.ByteRanges
import scala.concurrent.Promise import scala.concurrent.Promise
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.http.scaladsl.marshalling.ToResponseMarshaller import akka.http.scaladsl.marshalling.ToResponseMarshaller
import akka.http.scaladsl.unmarshalling.{ Unmarshaller, FromRequestUnmarshaller } import akka.http.scaladsl.unmarshalling.{ FromEntityUnmarshaller, MultipartUnmarshallers, Unmarshaller, FromRequestUnmarshaller }
import akka.http.impl.util._ import akka.http.impl.util._
trait MarshallingDirectives { trait MarshallingDirectives {

View file

@ -4,12 +4,15 @@
package akka.http.scaladsl.unmarshalling package akka.http.scaladsl.unmarshalling
import akka.actor.ActorSystem
import akka.http.ParserSettings
import scala.collection.immutable import scala.collection.immutable
import scala.collection.immutable.VectorBuilder import scala.collection.immutable.VectorBuilder
import akka.util.ByteString import akka.util.ByteString
import akka.event.{ NoLogging, LoggingAdapter } import akka.event.{ NoLogging, LoggingAdapter }
import akka.stream.OverflowStrategy import akka.stream.{ActorMaterializer, OverflowStrategy}
import akka.stream.impl.fusing.{ GraphInterpreter, IteratorInterpreter } import akka.stream.impl.fusing.IteratorInterpreter
import akka.stream.scaladsl._ import akka.stream.scaladsl._
import akka.http.impl.engine.parsing.BodyPartParser import akka.http.impl.engine.parsing.BodyPartParser
import akka.http.impl.util._ import akka.http.impl.util._
@ -59,7 +62,7 @@ trait MultipartUnmarshallers {
createStreamed: (MediaType.Multipart, Source[BP, Any]) T, createStreamed: (MediaType.Multipart, Source[BP, Any]) T,
createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS, createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS,
createStrict: (MediaType.Multipart, immutable.Seq[BPS]) T)(implicit log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] = createStrict: (MediaType.Multipart, immutable.Seq[BPS]) T)(implicit log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] =
Unmarshaller { implicit ec Unmarshaller.withMaterializer { implicit ec mat =>
entity entity
if (entity.contentType.mediaType.isMultipart && mediaRange.matches(entity.contentType.mediaType)) { if (entity.contentType.mediaType.isMultipart && mediaRange.matches(entity.contentType.mediaType)) {
entity.contentType.mediaType.params.get("boundary") match { entity.contentType.mediaType.params.get("boundary") match {
@ -67,7 +70,7 @@ trait MultipartUnmarshallers {
FastFuture.failed(new RuntimeException("Content-Type with a multipart media type must have a 'boundary' parameter")) FastFuture.failed(new RuntimeException("Content-Type with a multipart media type must have a 'boundary' parameter"))
case Some(boundary) case Some(boundary)
import BodyPartParser._ import BodyPartParser._
val parser = new BodyPartParser(defaultContentType, boundary, log) val parser = new BodyPartParser(defaultContentType, boundary, log, ParserSettings(ActorMaterializer.downcast(mat).system)) // TODO we could cache it
FastFuture.successful { FastFuture.successful {
entity match { entity match {
case HttpEntity.Strict(ContentType(mediaType: MediaType.Multipart, _), data) case HttpEntity.Strict(ContentType(mediaType: MediaType.Multipart, _), data)