!htc #19514,#18958 make config in HTTP non-case class + from reference
This commit is contained in:
parent
97df7bf689
commit
9b4b68f0c9
17 changed files with 215 additions and 124 deletions
|
|
@ -79,6 +79,17 @@ In Akka 2.4.x this is formulated like so:
|
|||
|
||||
.. 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
|
||||
=======================
|
||||
|
||||
|
|
|
|||
|
|
@ -19,14 +19,14 @@ import akka.http.impl.util._
|
|||
|
||||
import akka.http.scaladsl.model.headers.`User-Agent`
|
||||
|
||||
final case class ClientConnectionSettings(
|
||||
userAgentHeader: Option[`User-Agent`],
|
||||
connectingTimeout: FiniteDuration,
|
||||
idleTimeout: Duration,
|
||||
requestHeaderSizeHint: Int,
|
||||
websocketRandomFactory: () ⇒ Random,
|
||||
socketOptions: immutable.Traversable[SocketOption],
|
||||
parserSettings: ParserSettings) {
|
||||
final class ClientConnectionSettings(
|
||||
val userAgentHeader: Option[`User-Agent`],
|
||||
val connectingTimeout: FiniteDuration,
|
||||
val idleTimeout: Duration,
|
||||
val requestHeaderSizeHint: Int,
|
||||
val websocketRandomFactory: () ⇒ Random,
|
||||
val socketOptions: immutable.Traversable[SocketOption],
|
||||
val parserSettings: ParserSettings) {
|
||||
|
||||
require(connectingTimeout >= Duration.Zero, "connectingTimeout 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") {
|
||||
def fromSubConfig(root: Config, inner: Config) = {
|
||||
val c = inner.withFallback(root.getConfig(prefix))
|
||||
apply(
|
||||
c.getString("user-agent-header").toOption.map(`User-Agent`(_)),
|
||||
c getFiniteDuration "connecting-timeout",
|
||||
c getPotentiallyInfiniteDuration "idle-timeout",
|
||||
c getIntBytes "request-header-size-hint",
|
||||
Randoms.SecureRandomInstances, // can currently only be overridden from code
|
||||
SocketOptionSettings.fromSubConfig(root, c.getConfig("socket-options")),
|
||||
ParserSettings.fromSubConfig(root, c.getConfig("parsing")))
|
||||
new ClientConnectionSettings(
|
||||
userAgentHeader = c.getString("user-agent-header").toOption.map(`User-Agent`(_)),
|
||||
connectingTimeout = c getFiniteDuration "connecting-timeout",
|
||||
idleTimeout = c getPotentiallyInfiniteDuration "idle-timeout",
|
||||
requestHeaderSizeHint = c getIntBytes "request-header-size-hint",
|
||||
websocketRandomFactory = Randoms.SecureRandomInstances, // can currently only be overridden from code
|
||||
socketOptions = SocketOptionSettings.fromSubConfig(root, c.getConfig("socket-options")),
|
||||
parserSettings = ParserSettings.fromSubConfig(root, c.getConfig("parsing")))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -27,13 +27,13 @@ object ConnectionPoolSetup {
|
|||
ConnectionPoolSetup(settings, connectionContext, log)
|
||||
}
|
||||
|
||||
final case class ConnectionPoolSettings(
|
||||
maxConnections: Int,
|
||||
maxRetries: Int,
|
||||
maxOpenRequests: Int,
|
||||
pipeliningLimit: Int,
|
||||
idleTimeout: Duration,
|
||||
connectionSettings: ClientConnectionSettings) {
|
||||
final class ConnectionPoolSettings(
|
||||
val maxConnections: Int,
|
||||
val maxRetries: Int,
|
||||
val maxOpenRequests: Int,
|
||||
val pipeliningLimit: Int,
|
||||
val idleTimeout: Duration,
|
||||
val connectionSettings: ClientConnectionSettings) {
|
||||
|
||||
require(maxConnections > 0, "max-connections 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") {
|
||||
def fromSubConfig(root: Config, c: Config) = {
|
||||
apply(
|
||||
new ConnectionPoolSettings(
|
||||
c getInt "max-connections",
|
||||
c getInt "max-retries",
|
||||
c getInt "max-open-requests",
|
||||
|
|
|
|||
|
|
@ -4,32 +4,33 @@
|
|||
|
||||
package akka.http
|
||||
|
||||
import java.util.Locale
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.{ ActorRefFactory, ActorSystem }
|
||||
import akka.stream.{ ActorMaterializer, Materializer }
|
||||
import com.typesafe.config.Config
|
||||
import scala.collection.JavaConverters._
|
||||
import akka.http.scaladsl.model.{ StatusCode, HttpMethod, Uri }
|
||||
import akka.http.impl.util._
|
||||
import akka.http.impl.engine.parsing.HttpHeaderParser
|
||||
import akka.http.impl.engine.parsing.{ BodyPartParser, HttpHeaderParser }
|
||||
|
||||
final case class ParserSettings(
|
||||
maxUriLength: Int,
|
||||
maxMethodLength: Int,
|
||||
maxResponseReasonLength: Int,
|
||||
maxHeaderNameLength: Int,
|
||||
maxHeaderValueLength: Int,
|
||||
maxHeaderCount: Int,
|
||||
maxContentLength: Long,
|
||||
maxChunkExtLength: Int,
|
||||
maxChunkSize: Int,
|
||||
uriParsingMode: Uri.ParsingMode,
|
||||
cookieParsingMode: ParserSettings.CookieParsingMode,
|
||||
illegalHeaderWarnings: Boolean,
|
||||
errorLoggingVerbosity: ParserSettings.ErrorLoggingVerbosity,
|
||||
headerValueCacheLimits: Map[String, Int],
|
||||
includeTlsSessionInfoHeader: Boolean,
|
||||
customMethods: String ⇒ Option[HttpMethod],
|
||||
customStatusCodes: Int ⇒ Option[StatusCode]) extends HttpHeaderParser.Settings {
|
||||
final class ParserSettings(
|
||||
val maxUriLength: Int,
|
||||
val maxMethodLength: Int,
|
||||
val maxResponseReasonLength: Int,
|
||||
val maxHeaderNameLength: Int,
|
||||
val maxHeaderValueLength: Int,
|
||||
val maxHeaderCount: Int,
|
||||
val maxContentLength: Long,
|
||||
val maxChunkExtLength: Int,
|
||||
val maxChunkSize: Int,
|
||||
val uriParsingMode: Uri.ParsingMode,
|
||||
val cookieParsingMode: ParserSettings.CookieParsingMode,
|
||||
val illegalHeaderWarnings: Boolean,
|
||||
val errorLoggingVerbosity: ParserSettings.ErrorLoggingVerbosity,
|
||||
val headerValueCacheLimits: Map[String, Int],
|
||||
val includeTlsSessionInfoHeader: Boolean,
|
||||
val customMethods: String ⇒ Option[HttpMethod],
|
||||
val customStatusCodes: Int ⇒ Option[StatusCode])
|
||||
extends BodyPartParser.Settings {
|
||||
|
||||
require(maxUriLength > 0, "max-uri-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(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)
|
||||
|
||||
def withCustomMethods(methods: HttpMethod*): ParserSettings = {
|
||||
|
|
@ -54,6 +55,41 @@ final case class ParserSettings(
|
|||
val map = codes.map(c ⇒ c.intValue -> c).toMap
|
||||
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") {
|
||||
|
|
@ -61,7 +97,7 @@ object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsi
|
|||
val c = inner.withFallback(root.getConfig(prefix))
|
||||
val cacheConfig = c getConfig "header-cache"
|
||||
|
||||
apply(
|
||||
new ParserSettings(
|
||||
c getIntBytes "max-uri-length",
|
||||
c getIntBytes "max-method-length",
|
||||
c getIntBytes "max-response-reason-length",
|
||||
|
|
@ -130,5 +166,8 @@ object ParserSettings extends SettingsCompanion[ParserSettings]("akka.http.parsi
|
|||
* Java API
|
||||
*/
|
||||
def create(configOverrides: String): ParserSettings = ParserSettings(configOverrides)
|
||||
|
||||
implicit def default(implicit refFactory: ActorRefFactory): ParserSettings =
|
||||
apply(actorSystem)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -22,21 +22,21 @@ import akka.http.impl.util._
|
|||
import akka.http.scaladsl.model.HttpHeader
|
||||
import akka.http.scaladsl.model.headers.{ Host, Server }
|
||||
|
||||
final case class ServerSettings(
|
||||
serverHeader: Option[Server],
|
||||
timeouts: ServerSettings.Timeouts,
|
||||
maxConnections: Int,
|
||||
pipeliningLimit: Int,
|
||||
remoteAddressHeader: Boolean,
|
||||
rawRequestUriHeader: Boolean,
|
||||
transparentHeadRequests: Boolean,
|
||||
verboseErrorMessages: Boolean,
|
||||
responseHeaderSizeHint: Int,
|
||||
backlog: Int,
|
||||
socketOptions: immutable.Traversable[SocketOption],
|
||||
defaultHostHeader: Host,
|
||||
websocketRandomFactory: () ⇒ Random,
|
||||
parserSettings: ParserSettings) {
|
||||
final class ServerSettings(
|
||||
val serverHeader: Option[Server],
|
||||
val timeouts: ServerSettings.Timeouts,
|
||||
val maxConnections: Int,
|
||||
val pipeliningLimit: Int,
|
||||
val remoteAddressHeader: Boolean,
|
||||
val rawRequestUriHeader: Boolean,
|
||||
val transparentHeadRequests: Boolean,
|
||||
val verboseErrorMessages: Boolean,
|
||||
val responseHeaderSizeHint: Int,
|
||||
val backlog: Int,
|
||||
val socketOptions: immutable.Traversable[SocketOption],
|
||||
val defaultHostHeader: Host,
|
||||
val websocketRandomFactory: () ⇒ Random,
|
||||
val parserSettings: ParserSettings) {
|
||||
|
||||
require(0 < maxConnections, "max-connections must be > 0")
|
||||
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") {
|
||||
final case class Timeouts(idleTimeout: Duration,
|
||||
requestTimeout: Duration,
|
||||
bindTimeout: FiniteDuration) {
|
||||
final class Timeouts(
|
||||
val idleTimeout: Duration,
|
||||
val requestTimeout: Duration,
|
||||
val bindTimeout: FiniteDuration) {
|
||||
require(idleTimeout > Duration.Zero, "idleTimeout must be infinite or > 0")
|
||||
require(requestTimeout > Duration.Zero, "requestTimeout must be infinite or > 0")
|
||||
require(bindTimeout > Duration.Zero, "bindTimeout must be > 0")
|
||||
}
|
||||
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(_)),
|
||||
Timeouts(
|
||||
new Timeouts(
|
||||
c getPotentiallyInfiniteDuration "idle-timeout",
|
||||
c getPotentiallyInfiniteDuration "request-timeout",
|
||||
c getFiniteDuration "bind-timeout"),
|
||||
|
|
|
|||
|
|
@ -5,8 +5,6 @@
|
|||
package akka.http.impl.engine.parsing
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.http.ParserSettings
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import scala.annotation.tailrec
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.parboiled2.CharPredicate
|
||||
|
|
@ -27,7 +25,7 @@ import akka.stream.impl.fusing.SubSource
|
|||
private[http] final class BodyPartParser(defaultContentType: ContentType,
|
||||
boundary: String,
|
||||
log: LoggingAdapter,
|
||||
settings: BodyPartParser.Settings = BodyPartParser.defaultSettings)
|
||||
settings: BodyPartParser.Settings)
|
||||
extends PushPullStage[ByteString, BodyPartParser.Output] {
|
||||
import BodyPartParser._
|
||||
import settings._
|
||||
|
|
@ -277,28 +275,9 @@ private[http] object BodyPartParser {
|
|||
final case class EntityPart(data: ByteString) extends Output
|
||||
final case class ParseError(info: ErrorInfo) extends PartStart
|
||||
|
||||
final case class Settings(
|
||||
maxHeaderNameLength: Int,
|
||||
maxHeaderValueLength: Int,
|
||||
maxHeaderCount: 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
|
||||
abstract class Settings extends HttpHeaderParser.Settings {
|
||||
def maxHeaderCount: Int
|
||||
def illegalHeaderWarnings: Boolean
|
||||
def defaultHeaderValueCacheLimit: Int
|
||||
}
|
||||
|
||||
// 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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -408,7 +408,7 @@ private[engine] final class HttpHeaderParser private (
|
|||
private[http] object HttpHeaderParser {
|
||||
import SpecializedHeaderValueParsers._
|
||||
|
||||
trait Settings extends HeaderParser.Settings {
|
||||
abstract class Settings extends HeaderParser.Settings {
|
||||
def maxHeaderNameLength: Int
|
||||
def maxHeaderValueLength: Int
|
||||
def headerValueCacheLimit(headerName: String): Int
|
||||
|
|
|
|||
|
|
@ -156,7 +156,7 @@ private[http] object HeaderParser {
|
|||
"www-authenticate",
|
||||
"x-forwarded-for")
|
||||
|
||||
trait Settings {
|
||||
abstract class Settings {
|
||||
def uriParsingMode: Uri.ParsingMode
|
||||
def cookieParsingMode: ParserSettings.CookieParsingMode
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.http.scaladsl.server
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import akka.http.ParserSettings
|
||||
import akka.stream.Materializer
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.http.scaladsl.marshalling.ToResponseMarshallable
|
||||
|
|
@ -42,6 +43,11 @@ trait RequestContext {
|
|||
*/
|
||||
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.
|
||||
*/
|
||||
|
|
@ -91,7 +97,12 @@ trait RequestContext {
|
|||
/**
|
||||
* 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.
|
||||
|
|
|
|||
|
|
@ -5,8 +5,9 @@
|
|||
package akka.http.scaladsl.server
|
||||
|
||||
import scala.concurrent.{ Future, ExecutionContextExecutor }
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.{ ActorMaterializer, Materializer }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.http.ParserSettings
|
||||
import akka.http.scaladsl.marshalling.{ Marshal, ToResponseMarshallable }
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
|
|
@ -21,13 +22,17 @@ private[http] class RequestContextImpl(
|
|||
val executionContext: ExecutionContextExecutor,
|
||||
val materializer: Materializer,
|
||||
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) =
|
||||
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 =
|
||||
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] =
|
||||
trm(request)(executionContext)
|
||||
|
|
@ -56,8 +61,11 @@ private[http] class RequestContextImpl(
|
|||
override def withLog(log: LoggingAdapter): RequestContext =
|
||||
if (log != this.log) copy(log = log) else this
|
||||
|
||||
override def withSettings(settings: RoutingSettings): RequestContext =
|
||||
if (settings != this.settings) copy(settings = settings) else this
|
||||
override def withRoutingSettings(routingSettings: RoutingSettings): RequestContext =
|
||||
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 =
|
||||
copy(request = f(request))
|
||||
|
|
@ -86,6 +94,7 @@ private[http] class RequestContextImpl(
|
|||
executionContext: ExecutionContextExecutor = executionContext,
|
||||
materializer: Materializer = materializer,
|
||||
log: LoggingAdapter = log,
|
||||
settings: RoutingSettings = settings) =
|
||||
new RequestContextImpl(request, unmatchedPath, executionContext, materializer, log, settings)
|
||||
routingSettings: RoutingSettings = settings,
|
||||
parserSettings: ParserSettings = parserSettings) =
|
||||
new RequestContextImpl(request, unmatchedPath, executionContext, materializer, log, routingSettings, parserSettings)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,7 +5,9 @@
|
|||
package akka.http.scaladsl.server
|
||||
|
||||
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 akka.stream.scaladsl.Flow
|
||||
|
|
@ -23,6 +25,7 @@ object Route {
|
|||
* "Seals" a route by wrapping it with exception handling and rejection conversion.
|
||||
*/
|
||||
def seal(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
parserSettings: ParserSettings = null,
|
||||
rejectionHandler: RejectionHandler = RejectionHandler.default,
|
||||
exceptionHandler: ExceptionHandler = null): Route = {
|
||||
import directives.ExecutionDirectives._
|
||||
|
|
@ -39,6 +42,7 @@ object Route {
|
|||
* This conversion is also implicitly available through [[RouteResult.route2HandlerFlow]].
|
||||
*/
|
||||
def handlerFlow(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
parserSettings: ParserSettings,
|
||||
materializer: Materializer,
|
||||
routingLog: RoutingLog,
|
||||
executionContext: ExecutionContextExecutor = null,
|
||||
|
|
@ -50,6 +54,7 @@ object Route {
|
|||
* Turns a `Route` into an async handler function.
|
||||
*/
|
||||
def asyncHandler(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
parserSettings: ParserSettings,
|
||||
materializer: Materializer,
|
||||
routingLog: RoutingLog,
|
||||
executionContext: ExecutionContextExecutor = null,
|
||||
|
|
@ -59,10 +64,11 @@ object Route {
|
|||
|
||||
{
|
||||
implicit val executionContext = effectiveEC // overrides parameter
|
||||
val effectiveParserSettings = if (parserSettings ne null) parserSettings else ParserSettings(ActorMaterializer.downcast(materializer).system)
|
||||
|
||||
val sealedRoute = seal(route)
|
||||
request ⇒
|
||||
sealedRoute(new RequestContextImpl(request, routingLog.requestLog(request), routingSettings)).fast
|
||||
sealedRoute(new RequestContextImpl(request, routingLog.requestLog(request), routingSettings, effectiveParserSettings)).fast
|
||||
.map {
|
||||
case RouteResult.Complete(response) ⇒ response
|
||||
case RouteResult.Rejected(rejected) ⇒ throw new IllegalStateException(s"Unhandled rejections '$rejected', unsealed RejectionHandler?!")
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package akka.http.scaladsl.server
|
||||
|
||||
import akka.NotUsed
|
||||
import akka.http.ParserSettings
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.ExecutionContext
|
||||
|
|
@ -25,6 +26,7 @@ object RouteResult {
|
|||
final case class Rejected(rejections: immutable.Seq[Rejection]) extends RouteResult
|
||||
|
||||
implicit def route2HandlerFlow(route: Route)(implicit routingSettings: RoutingSettings,
|
||||
parserSettings: ParserSettings,
|
||||
materializer: Materializer,
|
||||
routingLog: RoutingLog,
|
||||
executionContext: ExecutionContext = null,
|
||||
|
|
|
|||
|
|
@ -8,17 +8,35 @@ import com.typesafe.config.Config
|
|||
import akka.actor.ActorRefFactory
|
||||
import akka.http.impl.util._
|
||||
|
||||
case class RoutingSettings(
|
||||
verboseErrorMessages: Boolean,
|
||||
fileGetConditional: Boolean,
|
||||
renderVanityFooter: Boolean,
|
||||
rangeCountLimit: Int,
|
||||
rangeCoalescingThreshold: Long,
|
||||
decodeMaxBytesPerChunk: Int,
|
||||
fileIODispatcher: String)
|
||||
final class RoutingSettings(
|
||||
val verboseErrorMessages: Boolean,
|
||||
val fileGetConditional: Boolean,
|
||||
val renderVanityFooter: Boolean,
|
||||
val rangeCountLimit: Int,
|
||||
val rangeCoalescingThreshold: Long,
|
||||
val decodeMaxBytesPerChunk: Int,
|
||||
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") {
|
||||
def fromSubConfig(root: Config, c: Config) = apply(
|
||||
def fromSubConfig(root: Config, c: Config) = new RoutingSettings(
|
||||
c getBoolean "verbose-error-messages",
|
||||
c getBoolean "file-get-conditional",
|
||||
c getBoolean "render-vanity-footer",
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import scala.concurrent.{ Future, ExecutionContextExecutor }
|
|||
import scala.collection.immutable
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.Materializer
|
||||
import akka.http.ParserSettings
|
||||
import akka.http.scaladsl.server.util.Tuple
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
import akka.http.scaladsl.model._
|
||||
|
|
@ -167,13 +168,13 @@ trait BasicDirectives {
|
|||
* Runs its inner route with the given alternative [[RoutingSettings]].
|
||||
*/
|
||||
def withSettings(settings: RoutingSettings): Directive0 =
|
||||
mapRequestContext(_ withSettings settings)
|
||||
mapRequestContext(_ withRoutingSettings settings)
|
||||
|
||||
/**
|
||||
* Runs the inner route with settings mapped by the given function.
|
||||
*/
|
||||
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]].
|
||||
|
|
@ -181,6 +182,12 @@ trait BasicDirectives {
|
|||
def extractSettings: Directive1[RoutingSettings] =
|
||||
BasicDirectives._extractSettings
|
||||
|
||||
/**
|
||||
* Extracts the [[akka.http.ParserSettings]] from the [[RequestContext]].
|
||||
*/
|
||||
def extractParserSettings: Directive1[ParserSettings] =
|
||||
BasicDirectives._extractParserSettings
|
||||
|
||||
/**
|
||||
* Extracts the [[RequestContext]] itself.
|
||||
*/
|
||||
|
|
@ -195,5 +202,6 @@ object BasicDirectives extends BasicDirectives {
|
|||
private val _extractMaterializer: Directive1[Materializer] = extract(_.materializer)
|
||||
private val _extractLog: Directive1[LoggingAdapter] = extract(_.log)
|
||||
private val _extractSettings: Directive1[RoutingSettings] = extract(_.settings)
|
||||
private val _extractParserSettings: Directive1[ParserSettings] = extract(_.parserSettings)
|
||||
private val _extractRequestContext: Directive1[RequestContext] = extract(conforms)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -76,7 +76,6 @@ trait FileUploadDirectives {
|
|||
case Some(tuple) ⇒ provide(tuple)
|
||||
case None ⇒ reject(MissingFormFieldRejection(fieldName))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object FileUploadDirectives extends FileUploadDirectives
|
||||
|
|
|
|||
|
|
@ -5,10 +5,15 @@
|
|||
package akka.http.scaladsl.server
|
||||
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.util.{ Failure, Success }
|
||||
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._
|
||||
|
||||
trait MarshallingDirectives {
|
||||
|
|
|
|||
|
|
@ -4,12 +4,15 @@
|
|||
|
||||
package akka.http.scaladsl.unmarshalling
|
||||
|
||||
import akka.actor.ActorSystem
|
||||
import akka.http.ParserSettings
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.immutable.VectorBuilder
|
||||
import akka.util.ByteString
|
||||
import akka.event.{ NoLogging, LoggingAdapter }
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.impl.fusing.{ GraphInterpreter, IteratorInterpreter }
|
||||
import akka.stream.{ActorMaterializer, OverflowStrategy}
|
||||
import akka.stream.impl.fusing.IteratorInterpreter
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.impl.engine.parsing.BodyPartParser
|
||||
import akka.http.impl.util._
|
||||
|
|
@ -59,7 +62,7 @@ trait MultipartUnmarshallers {
|
|||
createStreamed: (MediaType.Multipart, Source[BP, Any]) ⇒ T,
|
||||
createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) ⇒ BPS,
|
||||
createStrict: (MediaType.Multipart, immutable.Seq[BPS]) ⇒ T)(implicit log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] =
|
||||
Unmarshaller { implicit ec ⇒
|
||||
Unmarshaller.withMaterializer { implicit ec ⇒ mat =>
|
||||
entity ⇒
|
||||
if (entity.contentType.mediaType.isMultipart && mediaRange.matches(entity.contentType.mediaType)) {
|
||||
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"))
|
||||
case Some(boundary) ⇒
|
||||
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 {
|
||||
entity match {
|
||||
case HttpEntity.Strict(ContentType(mediaType: MediaType.Multipart, _), data) ⇒
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue