fusable SplitWhen and reusable HTTP blueprint

plus further optimizations

Lots of contributions from drewhk (Endre Varga).
This commit is contained in:
Roland Kuhn 2015-12-20 12:54:05 +01:00
parent 3c3545020b
commit 469e15581f
45 changed files with 1251 additions and 838 deletions

View file

@ -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 = {

View file

@ -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 = {

View file

@ -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] {

View file

@ -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)
}

View file

@ -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(

View file

@ -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))

View file

@ -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 {
}
}
}
}
}

View file

@ -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()

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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] =

View file

@ -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

View file

@ -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()
}
}

View file

@ -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))