fusable SplitWhen and reusable HTTP blueprint
plus further optimizations Lots of contributions from drewhk (Endre Varga).
This commit is contained in:
parent
3c3545020b
commit
469e15581f
45 changed files with 1251 additions and 838 deletions
|
|
@ -4,6 +4,7 @@
|
|||
|
||||
package akka.http.impl.engine.client
|
||||
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import language.existentials
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
|
@ -22,6 +23,7 @@ import akka.http.impl.util._
|
|||
import akka.stream.stage.GraphStage
|
||||
import akka.stream.stage.GraphStageLogic
|
||||
import akka.stream.stage.InHandler
|
||||
import akka.stream.impl.fusing.SubSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -69,11 +71,21 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
.mapConcat(conforms)
|
||||
.splitWhen(x ⇒ x.isInstanceOf[MessageStart] || x == MessageEnd)
|
||||
.prefixAndTail(1)
|
||||
.collect {
|
||||
.filter {
|
||||
case (Seq(MessageEnd), remaining) ⇒
|
||||
SubSource.kill(remaining)
|
||||
false
|
||||
case _ ⇒
|
||||
true
|
||||
}
|
||||
.map {
|
||||
case (Seq(ResponseStart(statusCode, protocol, headers, createEntity, _)), entityParts) ⇒
|
||||
val entity = createEntity(entityParts) withSizeLimit parserSettings.maxContentLength
|
||||
HttpResponse(statusCode, headers, entity, protocol)
|
||||
case (Seq(MessageStartError(_, info)), _) ⇒ throw IllegalResponseException(info)
|
||||
case (Seq(MessageStartError(_, info)), tail) ⇒
|
||||
// Tails can be empty, but still need one pull to figure that out -- never drop tails.
|
||||
SubSource.kill(tail)
|
||||
throw IllegalResponseException(info)
|
||||
}.concatSubstreams
|
||||
|
||||
val core = BidiFlow.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
|
|
@ -198,7 +210,8 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
|
||||
val getNextData = () ⇒ {
|
||||
waitingForMethod = false
|
||||
pull(dataInput)
|
||||
if (!isClosed(dataInput)) pull(dataInput)
|
||||
else completeStage()
|
||||
}
|
||||
|
||||
@tailrec def drainParser(current: ResponseOutput, b: ListBuffer[ResponseOutput] = ListBuffer.empty): Unit = {
|
||||
|
|
|
|||
|
|
@ -5,18 +5,18 @@
|
|||
package akka.http.impl.engine.parsing
|
||||
|
||||
import akka.http.ParserSettings
|
||||
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import scala.annotation.tailrec
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.parboiled2.CharPredicate
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.http.impl.util._
|
||||
import headers._
|
||||
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import akka.stream.impl.fusing.SubSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -173,7 +173,11 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
|||
emit(bytes)
|
||||
},
|
||||
emitFinalPartChunk: (List[HttpHeader], ContentType, ByteString) ⇒ Unit = {
|
||||
(headers, ct, bytes) ⇒ emit(BodyPartStart(headers, _ ⇒ HttpEntity.Strict(ct, bytes)))
|
||||
(headers, ct, bytes) ⇒
|
||||
emit(BodyPartStart(headers, { rest ⇒
|
||||
SubSource.kill(rest)
|
||||
HttpEntity.Strict(ct, bytes)
|
||||
}))
|
||||
})(input: ByteString, offset: Int): StateResult =
|
||||
try {
|
||||
@tailrec def rec(index: Int): StateResult = {
|
||||
|
|
|
|||
|
|
@ -5,8 +5,10 @@
|
|||
package akka.http.impl.engine.parsing
|
||||
|
||||
import akka.http.scaladsl.model._
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import akka.stream.scaladsl.{ Sink, Source }
|
||||
import akka.util.ByteString
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.impl.fusing.SubSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -64,7 +66,11 @@ private[http] object ParserOutput {
|
|||
sealed abstract class EntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity] extends (Source[A, Unit] ⇒ B)
|
||||
|
||||
final case class StrictEntityCreator(entity: HttpEntity.Strict) extends EntityCreator[ParserOutput, HttpEntity.Strict] {
|
||||
def apply(parts: Source[ParserOutput, Unit]) = entity
|
||||
def apply(parts: Source[ParserOutput, Unit]) = {
|
||||
// We might need to drain stray empty tail streams which will be read by no one.
|
||||
SubSource.kill(parts)
|
||||
entity
|
||||
}
|
||||
}
|
||||
final case class StreamedEntityCreator[-A <: ParserOutput, +B >: HttpEntity.Strict <: HttpEntity](creator: Source[A, Unit] ⇒ B)
|
||||
extends EntityCreator[A, B] {
|
||||
|
|
|
|||
|
|
@ -6,6 +6,7 @@ package akka.http.impl.engine.server
|
|||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.Random
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import scala.collection.immutable
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
import scala.util.control.NonFatal
|
||||
|
|
@ -25,6 +26,8 @@ import akka.stream.io._
|
|||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
import akka.http.scaladsl.model.ws.Message
|
||||
import akka.stream.impl.fusing.SubSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -48,13 +51,13 @@ import akka.util.ByteString
|
|||
* +----------+ +-------------+ Context +-----------+
|
||||
*/
|
||||
private[http] object HttpServerBluePrint {
|
||||
def apply(settings: ServerSettings, remoteAddress: Option[InetSocketAddress], log: LoggingAdapter)(implicit mat: Materializer): Http.ServerLayer = {
|
||||
def apply(settings: ServerSettings, remoteAddress: Option[InetSocketAddress], log: LoggingAdapter): Http.ServerLayer = {
|
||||
val theStack =
|
||||
userHandlerGuard(settings.pipeliningLimit) atop
|
||||
requestPreparation(settings) atop
|
||||
controller(settings, log) atop
|
||||
parsingRendering(settings, log) atop
|
||||
websocketSupport(settings, log) atop
|
||||
new ProtocolSwitchStage(settings, log) atop
|
||||
unwrapTls
|
||||
|
||||
theStack.withAttributes(HttpAttributes.remoteAddress(remoteAddress))
|
||||
|
|
@ -63,28 +66,13 @@ private[http] object HttpServerBluePrint {
|
|||
val unwrapTls: BidiFlow[ByteString, SslTlsOutbound, SslTlsInbound, ByteString, Unit] =
|
||||
BidiFlow.fromFlows(Flow[ByteString].map(SendBytes), Flow[SslTlsInbound].collect { case x: SessionBytes ⇒ x.bytes })
|
||||
|
||||
/** Wrap an HTTP implementation with support for switching to Websocket */
|
||||
def websocketSupport(settings: ServerSettings, log: LoggingAdapter)(implicit mat: Materializer): BidiFlow[ResponseRenderingOutput, ByteString, ByteString, ByteString, Unit] = {
|
||||
val ws = websocketSetup
|
||||
|
||||
BidiFlow.fromGraph(GraphDSL.create() { implicit b ⇒
|
||||
import GraphDSL.Implicits._
|
||||
|
||||
val switch = b.add(new ProtocolSwitchStage(ws.installHandler, settings.websocketRandomFactory, log))
|
||||
|
||||
switch.toWs ~> ws.websocketFlow ~> switch.fromWs
|
||||
|
||||
BidiShape(switch.fromHttp, switch.toNet, switch.fromNet, switch.toHttp)
|
||||
})
|
||||
}
|
||||
|
||||
def parsingRendering(settings: ServerSettings, log: LoggingAdapter): BidiFlow[ResponseRenderingContext, ResponseRenderingOutput, ByteString, RequestOutput, Unit] =
|
||||
BidiFlow.fromFlows(rendering(settings, log), parsing(settings, log))
|
||||
|
||||
def controller(settings: ServerSettings, log: LoggingAdapter): BidiFlow[HttpResponse, ResponseRenderingContext, RequestOutput, RequestOutput, Unit] =
|
||||
BidiFlow.fromGraph(new ControllerStage(settings, log)).reversed
|
||||
|
||||
def requestPreparation(settings: ServerSettings)(implicit mat: Materializer): BidiFlow[HttpResponse, HttpResponse, RequestOutput, HttpRequest, Unit] =
|
||||
def requestPreparation(settings: ServerSettings): BidiFlow[HttpResponse, HttpResponse, RequestOutput, HttpRequest, Unit] =
|
||||
BidiFlow.fromFlows(Flow[HttpResponse],
|
||||
Flow[RequestOutput]
|
||||
.splitWhen(x ⇒ x.isInstanceOf[MessageStart] || x == MessageEnd)
|
||||
|
|
@ -93,7 +81,7 @@ private[http] object HttpServerBluePrint {
|
|||
.concatSubstreams
|
||||
.via(requestStartOrRunIgnore(settings)))
|
||||
|
||||
def requestStartOrRunIgnore(settings: ServerSettings)(implicit mat: Materializer): Flow[(ParserOutput.RequestOutput, Source[ParserOutput.RequestOutput, Unit]), HttpRequest, Unit] =
|
||||
def requestStartOrRunIgnore(settings: ServerSettings): Flow[(ParserOutput.RequestOutput, Source[ParserOutput.RequestOutput, Unit]), HttpRequest, Unit] =
|
||||
Flow.fromGraph(new GraphStage[FlowShape[(RequestOutput, Source[RequestOutput, Unit]), HttpRequest]] {
|
||||
val in = Inlet[(RequestOutput, Source[RequestOutput, Unit])]("RequestStartThenRunIgnore.in")
|
||||
val out = Outlet[HttpRequest]("RequestStartThenRunIgnore.out")
|
||||
|
|
@ -115,7 +103,7 @@ private[http] object HttpServerBluePrint {
|
|||
push(out, HttpRequest(effectiveMethod, uri, effectiveHeaders, entity, protocol))
|
||||
|
||||
case (wat, src) ⇒
|
||||
src.runWith(Sink.ignore)
|
||||
SubSource.kill(src)
|
||||
pull(in)
|
||||
}
|
||||
})
|
||||
|
|
@ -355,124 +343,111 @@ private[http] object HttpServerBluePrint {
|
|||
def userHandlerGuard(pipeliningLimit: Int): BidiFlow[HttpResponse, HttpResponse, HttpRequest, HttpRequest, Unit] =
|
||||
One2OneBidiFlow[HttpRequest, HttpResponse](pipeliningLimit).reversed
|
||||
|
||||
private trait WebsocketSetup {
|
||||
def websocketFlow: Flow[ByteString, ByteString, Any]
|
||||
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit
|
||||
}
|
||||
private def websocketSetup: WebsocketSetup = {
|
||||
val sinkCell = new StreamUtils.OneTimeWriteCell[Publisher[FrameEvent]]
|
||||
val sourceCell = new StreamUtils.OneTimeWriteCell[Subscriber[FrameEvent]]
|
||||
private class ProtocolSwitchStage(settings: ServerSettings, log: LoggingAdapter)
|
||||
extends GraphStage[BidiShape[ResponseRenderingOutput, ByteString, ByteString, ByteString]] {
|
||||
|
||||
val sink = StreamUtils.oneTimePublisherSink[FrameEvent](sinkCell, "frameHandler.in")
|
||||
val source = StreamUtils.oneTimeSubscriberSource[FrameEvent](sourceCell, "frameHandler.out")
|
||||
|
||||
val flow = Websocket.framing.join(Flow.fromSinkAndSourceMat(sink, source)(Keep.none))
|
||||
|
||||
new WebsocketSetup {
|
||||
def websocketFlow: Flow[ByteString, ByteString, Any] = flow
|
||||
|
||||
def installHandler(handlerFlow: Flow[FrameEvent, FrameEvent, Any])(implicit mat: Materializer): Unit =
|
||||
Source.fromPublisher(sinkCell.value)
|
||||
.via(handlerFlow)
|
||||
.to(Sink.fromSubscriber(sourceCell.value))
|
||||
.run()
|
||||
}
|
||||
}
|
||||
|
||||
private case class ProtocolSwitchShape(
|
||||
fromNet: Inlet[ByteString],
|
||||
toNet: Outlet[ByteString],
|
||||
fromHttp: Inlet[ResponseRenderingOutput],
|
||||
toHttp: Outlet[ByteString],
|
||||
fromWs: Inlet[ByteString],
|
||||
toWs: Outlet[ByteString]) extends Shape {
|
||||
def inlets: immutable.Seq[Inlet[_]] = Vector(fromNet, fromHttp, fromWs)
|
||||
def outlets: immutable.Seq[Outlet[_]] = Vector(toNet, toHttp, toWs)
|
||||
|
||||
def deepCopy(): Shape =
|
||||
ProtocolSwitchShape(fromNet.carbonCopy(), toNet.carbonCopy(), fromHttp.carbonCopy(), toHttp.carbonCopy(), fromWs.carbonCopy(), toWs.carbonCopy())
|
||||
|
||||
def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = {
|
||||
require(inlets.size == 3 && outlets.size == 3, s"ProtocolSwitchShape must have 3 inlets and outlets but had ${inlets.size} / ${outlets.size}")
|
||||
ProtocolSwitchShape(
|
||||
inlets(0).asInstanceOf[Inlet[ByteString]],
|
||||
outlets(0).asInstanceOf[Outlet[ByteString]],
|
||||
inlets(1).asInstanceOf[Inlet[ResponseRenderingOutput]],
|
||||
outlets(1).asInstanceOf[Outlet[ByteString]],
|
||||
inlets(2).asInstanceOf[Inlet[ByteString]],
|
||||
outlets(2).asInstanceOf[Outlet[ByteString]])
|
||||
}
|
||||
}
|
||||
|
||||
private class ProtocolSwitchStage(installHandler: Flow[FrameEvent, FrameEvent, Any] ⇒ Unit,
|
||||
websocketRandomFactory: () ⇒ Random, log: LoggingAdapter) extends GraphStage[ProtocolSwitchShape] {
|
||||
private val fromNet = Inlet[ByteString]("fromNet")
|
||||
private val toNet = Outlet[ByteString]("toNet")
|
||||
|
||||
private val toHttp = Outlet[ByteString]("toHttp")
|
||||
private val fromHttp = Inlet[ResponseRenderingOutput]("fromHttp")
|
||||
|
||||
private val toWs = Outlet[ByteString]("toWs")
|
||||
private val fromWs = Inlet[ByteString]("fromWs")
|
||||
|
||||
override def initialAttributes = Attributes.name("ProtocolSwitchStage")
|
||||
|
||||
def shape: ProtocolSwitchShape = ProtocolSwitchShape(fromNet, toNet, fromHttp, toHttp, fromWs, toWs)
|
||||
override val shape = BidiShape(fromHttp, toNet, fromNet, toHttp)
|
||||
|
||||
def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
import akka.http.impl.engine.rendering.ResponseRenderingOutput._
|
||||
|
||||
var websocketHandlerWasInstalled = false
|
||||
setHandler(fromHttp, new InHandler {
|
||||
override def onPush(): Unit =
|
||||
grab(fromHttp) match {
|
||||
case HttpData(b) ⇒ push(toNet, b)
|
||||
case SwitchToWebsocket(bytes, handlerFlow) ⇒
|
||||
push(toNet, bytes)
|
||||
complete(toHttp)
|
||||
cancel(fromHttp)
|
||||
switchToWebsocket(handlerFlow)
|
||||
}
|
||||
})
|
||||
setHandler(toNet, new OutHandler {
|
||||
override def onPull(): Unit = pull(fromHttp)
|
||||
})
|
||||
|
||||
setHandler(fromHttp, ignoreTerminateInput)
|
||||
setHandler(toHttp, ignoreTerminateOutput)
|
||||
setHandler(fromWs, ignoreTerminateInput)
|
||||
setHandler(toWs, ignoreTerminateOutput)
|
||||
|
||||
val pullNet = () ⇒ pull(fromNet)
|
||||
setHandler(fromNet, new InHandler {
|
||||
def onPush(): Unit = emit(target, grab(fromNet), pullNet)
|
||||
def onPush(): Unit = push(toHttp, grab(fromNet))
|
||||
|
||||
// propagate error but don't close stage yet to prevent fromHttp/fromWs being cancelled
|
||||
// too eagerly
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = fail(target, ex)
|
||||
override def onUpstreamFailure(ex: Throwable): Unit = fail(toHttp, ex)
|
||||
})
|
||||
|
||||
val shutdown: () ⇒ Unit = () ⇒ completeStage()
|
||||
val httpToNet: ResponseRenderingOutput ⇒ Unit = {
|
||||
case HttpData(b) ⇒ push(toNet, b)
|
||||
case SwitchToWebsocket(bytes, handlerFlow) ⇒
|
||||
push(toNet, bytes)
|
||||
val frameHandler = handlerFlow match {
|
||||
case Left(frameHandler) ⇒ frameHandler
|
||||
case Right(messageHandler) ⇒
|
||||
Websocket.stack(serverSide = true, maskingRandomFactory = websocketRandomFactory, log = log).join(messageHandler)
|
||||
}
|
||||
installHandler(frameHandler)
|
||||
websocketHandlerWasInstalled = true
|
||||
}
|
||||
val wsToNet: ByteString ⇒ Unit = push(toNet, _)
|
||||
|
||||
setHandler(toNet, new OutHandler {
|
||||
def onPull(): Unit =
|
||||
if (isHttp) read(fromHttp)(httpToNet, shutdown)
|
||||
else read(fromWs)(wsToNet, shutdown)
|
||||
|
||||
// toNet cancellation isn't allowed to stop this stage
|
||||
setHandler(toHttp, new OutHandler {
|
||||
override def onPull(): Unit = pull(fromNet)
|
||||
override def onDownstreamFinish(): Unit = ()
|
||||
})
|
||||
|
||||
def isHttp = !websocketHandlerWasInstalled
|
||||
def isWS = websocketHandlerWasInstalled
|
||||
def target = if (websocketHandlerWasInstalled) toWs else toHttp
|
||||
private var activeTimers = 0
|
||||
private def timeout = ActorMaterializer.downcast(materializer).settings.subscriptionTimeoutSettings.timeout
|
||||
private def addTimeout(s: SubscriptionTimeout): Unit = {
|
||||
if (activeTimers == 0) setKeepGoing(true)
|
||||
activeTimers += 1
|
||||
scheduleOnce(s, timeout)
|
||||
}
|
||||
private def cancelTimeout(s: SubscriptionTimeout): Unit =
|
||||
if (isTimerActive(s)) {
|
||||
activeTimers -= 1
|
||||
if (activeTimers == 0) setKeepGoing(false)
|
||||
cancelTimer(s)
|
||||
}
|
||||
override def onTimer(timerKey: Any): Unit = timerKey match {
|
||||
case SubscriptionTimeout(f) ⇒
|
||||
activeTimers -= 1
|
||||
if (activeTimers == 0) setKeepGoing(false)
|
||||
f()
|
||||
}
|
||||
|
||||
override def preStart(): Unit = pull(fromNet)
|
||||
/*
|
||||
* Websocket support
|
||||
*/
|
||||
def switchToWebsocket(handlerFlow: Either[Flow[FrameEvent, FrameEvent, Any], Flow[Message, Message, Any]]): Unit = {
|
||||
val frameHandler = handlerFlow match {
|
||||
case Left(frameHandler) ⇒ frameHandler
|
||||
case Right(messageHandler) ⇒
|
||||
Websocket.stack(serverSide = true, maskingRandomFactory = settings.websocketRandomFactory, log = log).join(messageHandler)
|
||||
}
|
||||
val sinkIn = new SubSinkInlet[ByteString]("FrameSink")
|
||||
val sourceOut = new SubSourceOutlet[ByteString]("FrameSource")
|
||||
|
||||
override def postStop(): Unit = {
|
||||
// Install a dummy handler to make sure no processors leak because they have
|
||||
// never been subscribed to, see #17494 and #17551.
|
||||
if (!websocketHandlerWasInstalled) installHandler(Flow[FrameEvent])
|
||||
val timeoutKey = SubscriptionTimeout(() ⇒ {
|
||||
sourceOut.timeout(timeout)
|
||||
if (sourceOut.isClosed) completeStage()
|
||||
})
|
||||
addTimeout(timeoutKey)
|
||||
|
||||
sinkIn.setHandler(new InHandler {
|
||||
override def onPush(): Unit = push(toNet, sinkIn.grab())
|
||||
})
|
||||
setHandler(toNet, new OutHandler {
|
||||
override def onPull(): Unit = sinkIn.pull()
|
||||
})
|
||||
|
||||
setHandler(fromNet, new InHandler {
|
||||
override def onPush(): Unit = sourceOut.push(grab(fromNet))
|
||||
})
|
||||
sourceOut.setHandler(new OutHandler {
|
||||
override def onPull(): Unit = {
|
||||
if (!hasBeenPulled(fromNet)) pull(fromNet)
|
||||
cancelTimeout(timeoutKey)
|
||||
sourceOut.setHandler(new OutHandler {
|
||||
override def onPull(): Unit = pull(fromNet)
|
||||
})
|
||||
}
|
||||
})
|
||||
|
||||
Websocket.framing.join(frameHandler).runWith(sourceOut.source, sinkIn.sink)(subFusingMaterializer)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private case class SubscriptionTimeout(andThen: () ⇒ Unit)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,18 +5,16 @@
|
|||
package akka.http.impl.engine.ws
|
||||
|
||||
import java.util.Random
|
||||
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.stream._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
|
||||
import akka.http.impl.util._
|
||||
import akka.http.scaladsl.model.ws._
|
||||
import akka.stream.impl.fusing.SubSource
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -91,6 +89,7 @@ private[http] object Websocket {
|
|||
.map {
|
||||
case (seq, remaining) ⇒ seq.head match {
|
||||
case TextMessagePart(text, true) ⇒
|
||||
SubSource.kill(remaining)
|
||||
TextMessage.Strict(text)
|
||||
case first @ TextMessagePart(text, false) ⇒
|
||||
TextMessage(
|
||||
|
|
@ -99,6 +98,7 @@ private[http] object Websocket {
|
|||
case t: TextMessagePart if t.data.nonEmpty ⇒ t.data
|
||||
})
|
||||
case BinaryMessagePart(data, true) ⇒
|
||||
SubSource.kill(remaining)
|
||||
BinaryMessage.Strict(data)
|
||||
case first @ BinaryMessagePart(data, false) ⇒
|
||||
BinaryMessage(
|
||||
|
|
|
|||
|
|
@ -150,26 +150,6 @@ private[http] object StreamUtils {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source
|
||||
* will only be traversed once.
|
||||
*/
|
||||
def transformMultiple(input: Source[ByteString, Any], transformers: immutable.Seq[Flow[ByteString, ByteString, Any]])(implicit materializer: Materializer): immutable.Seq[Source[ByteString, Any]] =
|
||||
transformers match {
|
||||
case Nil ⇒ Nil
|
||||
case Seq(one) ⇒ Vector(input.via(one))
|
||||
case multiple ⇒
|
||||
val (fanoutSub, fanoutPub) = Source.asSubscriber[ByteString].toMat(Sink.asPublisher(true))(Keep.both).run()
|
||||
val sources = transformers.map { flow ⇒
|
||||
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources
|
||||
// are exposed, so there is no race
|
||||
Source.fromPublisher(Source.fromPublisher(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.asPublisher(false)))
|
||||
}
|
||||
// The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
|
||||
input.runWith(Sink.fromSubscriber(fanoutSub))
|
||||
sources
|
||||
}
|
||||
|
||||
def mapEntityError(f: Throwable ⇒ Throwable): RequestEntity ⇒ RequestEntity =
|
||||
_.transformDataBytes(mapErrorTransformer(f))
|
||||
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.event.{ NoLogging, LoggingAdapter }
|
|||
import akka.stream.impl.ConstantFun
|
||||
import akka.stream.Materializer
|
||||
import akka.stream.javadsl.{ Source ⇒ JSource }
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl._
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
import akka.http.scaladsl.model.headers._
|
||||
import akka.http.impl.engine.rendering.BodyPartRenderer
|
||||
|
|
@ -187,10 +187,7 @@ object Multipart {
|
|||
|
||||
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Any])(f: BP ⇒ Future[BPS])(implicit fm: Materializer): Future[Vector[BPS]] = {
|
||||
import fm.executionContext
|
||||
// TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed
|
||||
parts.runFold(new VectorBuilder[Future[BPS]]) {
|
||||
case (builder, part) ⇒ builder += f(part)
|
||||
}.fast.flatMap(builder ⇒ FastFuture.sequence(builder.result()))
|
||||
parts.mapAsync(Int.MaxValue)(f).runWith(Sink.seq).fast.map(_.toVector)
|
||||
}
|
||||
|
||||
//////////////////////// CONCRETE multipart types /////////////////////////
|
||||
|
|
@ -574,4 +571,4 @@ object Multipart {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ import akka.http.scaladsl.model._
|
|||
class ConnectionPoolSpec extends AkkaSpec("""
|
||||
akka.loggers = []
|
||||
akka.loglevel = OFF
|
||||
akka.io.tcp.windows-connection-abort-workaround-enabled = auto
|
||||
akka.io.tcp.trace-logging = off""") {
|
||||
implicit val materializer = ActorMaterializer()
|
||||
|
||||
|
|
|
|||
|
|
@ -104,6 +104,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
sendWireData("0\n\n")
|
||||
sub.request(1)
|
||||
probe.expectNext(HttpEntity.LastChunk)
|
||||
sub.request(1)
|
||||
probe.expectComplete()
|
||||
|
||||
requestsSub.sendComplete()
|
||||
|
|
@ -165,6 +166,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
sendWireData("0\n\n")
|
||||
sub.request(1)
|
||||
probe.expectNext(HttpEntity.LastChunk)
|
||||
sub.request(1)
|
||||
probe.expectComplete()
|
||||
|
||||
// simulate that response is received before method bypass reaches response parser
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
|
|||
val timeout = Timeout(Span(3, Seconds))
|
||||
|
||||
"The client implementation" should {
|
||||
"not accept certificates signed by unknown CA" in EventFilter[SSLException](occurrences = 1).intercept {
|
||||
"not accept certificates signed by unknown CA" in {
|
||||
val pipe = pipeline(Http().defaultClientHttpsContext, hostname = "akka.example.org") // default context doesn't include custom CA
|
||||
|
||||
whenReady(pipe(HttpRequest(uri = "https://akka.example.org/")).failed, timeout) { e ⇒
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
|
||||
package akka.http.impl.engine.parsing
|
||||
|
||||
import akka.stream.impl.fusing.GraphInterpreter
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -23,7 +24,7 @@ import akka.http.scaladsl.model._
|
|||
import akka.http.scaladsl.model.headers._
|
||||
import akka.http.scaladsl.util.FastFuture
|
||||
import akka.http.scaladsl.util.FastFuture._
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.{ OverflowStrategy, ActorMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
import akka.util.ByteString
|
||||
|
||||
|
|
@ -480,7 +481,9 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
|||
case (Seq(RequestStart(method, uri, protocol, headers, createEntity, _, close)), entityParts) ⇒
|
||||
closeAfterResponseCompletion :+= close
|
||||
Right(HttpRequest(method, uri, headers, createEntity(entityParts), protocol))
|
||||
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), _) ⇒ Left(x)
|
||||
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), rest) ⇒
|
||||
rest.runWith(Sink.cancelled)
|
||||
Left(x)
|
||||
}
|
||||
.concatSubstreams
|
||||
.flatMapConcat { x ⇒
|
||||
|
|
|
|||
|
|
@ -298,7 +298,9 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
|||
case (Seq(ResponseStart(statusCode, protocol, headers, createEntity, close)), entityParts) ⇒
|
||||
closeAfterResponseCompletion :+= close
|
||||
Right(HttpResponse(statusCode, headers, createEntity(entityParts), protocol))
|
||||
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), _) ⇒ Left(x)
|
||||
case (Seq(x @ (MessageStartError(_, _) | EntityStreamError(_))), tail) ⇒
|
||||
tail.runWith(Sink.ignore)
|
||||
Left(x)
|
||||
}.concatSubstreams
|
||||
|
||||
def collectBlocking[T](source: Source[T, Any]): Seq[T] =
|
||||
|
|
|
|||
|
|
@ -66,6 +66,8 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
|
|
||||
|""")
|
||||
|
||||
requests.request(1)
|
||||
|
||||
expectResponseWithWipedDate(
|
||||
"""HTTP/1.1 505 HTTP Version Not Supported
|
||||
|Server: akka-http/test
|
||||
|
|
@ -504,6 +506,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
dataProbe.expectNoMsg(50.millis)
|
||||
send("0123456789ABCDEF")
|
||||
dataProbe.expectNext(ByteString("0123456789ABCDEF"))
|
||||
dataSub.request(1)
|
||||
dataProbe.expectComplete()
|
||||
responses.sendNext(HttpResponse(entity = "Yeah"))
|
||||
expectResponseWithWipedDate(
|
||||
|
|
@ -545,6 +548,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
|""")
|
||||
dataProbe.expectNext(Chunk(ByteString("0123456789ABCDEF")))
|
||||
dataProbe.expectNext(LastChunk)
|
||||
dataSub.request(1)
|
||||
dataProbe.expectComplete()
|
||||
responses.sendNext(HttpResponse(entity = "Yeah"))
|
||||
expectResponseWithWipedDate(
|
||||
|
|
@ -663,6 +667,8 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
|
|
||||
|""")
|
||||
|
||||
requests.request(1)
|
||||
|
||||
expectResponseWithWipedDate(
|
||||
"""|HTTP/1.1 400 Bad Request
|
||||
|Server: akka-http/test
|
||||
|
|
@ -701,6 +707,7 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
|
|||
|
||||
val HttpRequest(POST, _, _, entity, _) = expectRequest()
|
||||
responses.sendNext(HttpResponse(status = StatusCodes.InsufficientStorage))
|
||||
entity.dataBytes.runWith(Sink.ignore)
|
||||
|
||||
expectResponseWithWipedDate(
|
||||
"""HTTP/1.1 507 Insufficient Storage
|
||||
|
|
|
|||
|
|
@ -68,6 +68,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val data2 = ByteString("def", "ASCII")
|
||||
pushInput(data2)
|
||||
sub.expectNext(data2)
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
|
||||
|
|
@ -87,6 +88,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val data2 = ByteString("defg", "ASCII")
|
||||
pushInput(header2 ++ data2)
|
||||
sub.expectNext(data2)
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
"for several messages" in new ClientTestSetup {
|
||||
|
|
@ -107,6 +109,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val data3 = ByteString("h")
|
||||
pushInput(header2 ++ data2 ++ header3 ++ data3)
|
||||
sub.expectNext(data2)
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
|
||||
val dataSource2 = expectBinaryMessage().dataStream
|
||||
|
|
@ -119,6 +122,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val data4 = ByteString("i")
|
||||
pushInput(data4)
|
||||
sub2.expectNext(data4)
|
||||
s2.request(1)
|
||||
sub2.expectComplete()
|
||||
}
|
||||
"unmask masked input on the server side" in new ServerTestSetup {
|
||||
|
|
@ -138,6 +142,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
pushInput(data2)
|
||||
sub.expectNext(ByteString("def", "ASCII"))
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
"unmask masked input on the server side for empty frame" in new ServerTestSetup {
|
||||
|
|
@ -218,6 +223,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
pushInput(data2)
|
||||
sub.expectNext(ByteString("cdef€", "UTF-8"))
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
"unmask masked input on the server side for empty frame" in new ServerTestSetup {
|
||||
|
|
@ -430,6 +436,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
val input2 = frameHeader(Opcode.Continuation, 3, fin = true, mask = Some(mask2)) ++ maskedASCII("456", mask2)._1
|
||||
pushInput(input2)
|
||||
sub.expectNext(ByteString("456", "ASCII"))
|
||||
s.request(1)
|
||||
sub.expectComplete()
|
||||
}
|
||||
"don't respond to unsolicited pong frames" in new ClientTestSetup {
|
||||
|
|
@ -770,6 +777,13 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
pushInput(frameHeader(Opcode.Text, 0, fin = false))
|
||||
pushInput(frameHeader(Opcode.Continuation, 3, fin = true) ++ data)
|
||||
|
||||
// Kids, always drain your entities
|
||||
messageIn.requestNext() match {
|
||||
case b: TextMessage ⇒
|
||||
b.textStream.runWith(Sink.ignore)
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
expectError(messageIn)
|
||||
|
||||
expectCloseCodeOnNetwork(Protocol.CloseCodes.InconsistentData)
|
||||
|
|
@ -927,10 +941,12 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
def expectComplete[T](probe: TestSubscriber.Probe[T]): Unit = {
|
||||
probe.ensureSubscription()
|
||||
probe.request(1)
|
||||
probe.expectComplete()
|
||||
}
|
||||
def expectError[T](probe: TestSubscriber.Probe[T]): Throwable = {
|
||||
probe.ensureSubscription()
|
||||
probe.request(1)
|
||||
probe.expectError()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -33,6 +33,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll wit
|
|||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.loglevel = ERROR
|
||||
akka.stdout-loglevel = ERROR
|
||||
windows-connection-abort-workaround-enabled = auto
|
||||
akka.log-dead-letters = OFF
|
||||
""")
|
||||
implicit val system = ActorSystem(getClass.getSimpleName, testConf)
|
||||
|
|
@ -89,7 +90,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll wit
|
|||
}
|
||||
|
||||
"properly terminate client when server is not running" in Utils.assertAllStagesStopped {
|
||||
for (i ← 1 to 100)
|
||||
for (i ← 1 to 10)
|
||||
withClue(s"iterator $i: ") {
|
||||
Source.single(HttpRequest(HttpMethods.POST, "/test", List.empty, HttpEntity(MediaTypes.`text/plain`.withCharset(HttpCharsets.`UTF-8`), "buh")))
|
||||
.via(Http(actorSystem).outgoingConnection("localhost", 7777))
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue