Merge pull request #19640 from akka/wip-19623-early-response-errors-johanandren

=htt #19623 Early response no longer causing push or pull twice errors
This commit is contained in:
drewhk 2016-02-01 12:06:59 +01:00
commit 5e6b340dd1
2 changed files with 292 additions and 32 deletions

View file

@ -89,54 +89,106 @@ private[http] object HttpServerBluePrint {
case _ BidiFlow.identity
}
/**
* Two state stage, either transforms an incoming RequestOutput into a HttpRequest with strict entity and then pushes
* that (the "idle" inHandler) or creates a HttpRequest with a streamed entity and switch to a state which will push
* incoming chunks into the streaming entity until end of request is reached (the StreamedEntityCreator case in create
* entity).
*/
final class PrepareRequests(settings: ServerSettings) extends GraphStage[FlowShape[RequestOutput, HttpRequest]] {
val in = Inlet[RequestOutput]("RequestStartThenRunIgnore.in")
val out = Outlet[HttpRequest]("RequestStartThenRunIgnore.out")
override val shape: FlowShape[RequestOutput, HttpRequest] = FlowShape.of(in, out)
override def createLogic(inheritedAttributes: Attributes) = new GraphStageLogic(shape) {
override def createLogic(inheritedAttributes: Attributes) = new GraphStageLogic(shape) with InHandler with OutHandler {
val remoteAddress = inheritedAttributes.get[HttpAttributes.RemoteAddress].flatMap(_.address)
var downstreamPullWaiting = false
var completionDeferred = false
val idle = new InHandler {
def onPush(): Unit = grab(in) match {
case RequestStart(method, uri, protocol, hdrs, entityCreator, _, _)
val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method
val effectiveHeaders =
if (settings.remoteAddressHeader && remoteAddress.isDefined)
headers.`Remote-Address`(RemoteAddress(remoteAddress.get)) +: hdrs
else hdrs
// optimization: to avoid allocations the "idle" case in and out handlers are put directly on the GraphStageLogic itself
override def onPull(): Unit = {
pull(in)
}
val entity = createEntity(entityCreator) withSizeLimit settings.parserSettings.maxContentLength
push(out, HttpRequest(effectiveMethod, uri, effectiveHeaders, entity, protocol))
case other
throw new IllegalStateException(s"unexpected element of type ${other.getClass}")
override def onPush(): Unit = grab(in) match {
case RequestStart(method, uri, protocol, hdrs, entityCreator, _, _)
val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method
val effectiveHeaders =
if (settings.remoteAddressHeader && remoteAddress.isDefined)
headers.`Remote-Address`(RemoteAddress(remoteAddress.get)) +: hdrs
else hdrs
val entity = createEntity(entityCreator) withSizeLimit settings.parserSettings.maxContentLength
push(out, HttpRequest(effectiveMethod, uri, effectiveHeaders, entity, protocol))
case other
throw new IllegalStateException(s"unexpected element of type ${other.getClass}")
}
setIdleHandlers()
def setIdleHandlers() {
if (completionDeferred) {
completeStage()
} else {
setHandler(in, this)
setHandler(out, this)
if (downstreamPullWaiting) {
downstreamPullWaiting = false
pull(in)
}
}
}
setHandler(in, idle)
def createEntity(creator: EntityCreator[RequestOutput, RequestEntity]): RequestEntity =
creator match {
case StrictEntityCreator(entity) entity
case StreamedEntityCreator(creator)
val entitySource = new SubSourceOutlet[RequestOutput]("EntitySource")
entitySource.setHandler(new OutHandler {
def onPull(): Unit = pull(in)
})
setHandler(in, new InHandler {
def onPush(): Unit = grab(in) match {
case MessageEnd
entitySource.complete()
setHandler(in, idle)
case x entitySource.push(x)
}
override def onUpstreamFinish(): Unit = completeStage()
})
creator(Source.fromGraph(entitySource.source))
case StreamedEntityCreator(creator) streamRequestEntity(creator)
}
setHandler(out, new OutHandler {
override def onPull(): Unit = pull(in)
})
def streamRequestEntity(creator: (Source[ParserOutput.RequestOutput, NotUsed]) => RequestEntity): RequestEntity = {
// stream incoming chunks into the request entity until we reach the end of it
// and then toggle back to "idle"
val entitySource = new SubSourceOutlet[RequestOutput]("EntitySource")
// optimization: re-use the idle outHandler
entitySource.setHandler(this)
setHandler(in, new InHandler {
def onPush(): Unit = {
grab(in) match {
case MessageEnd
entitySource.complete()
setIdleHandlers()
case x entitySource.push(x)
}
}
override def onUpstreamFinish(): Unit = {
entitySource.complete()
completeStage()
}
override def onUpstreamFailure(ex: Throwable): Unit = {
entitySource.fail(ex)
failStage(ex)
}
})
setHandler(out, new OutHandler {
override def onPull(): Unit = {
// remember this until we are done with the chunked entity
// so can pull downstream then
downstreamPullWaiting = true
}
override def onDownstreamFinish(): Unit = {
// downstream signalled not wanting any more requests
// we should keep processing the entity stream and then
// when it completes complete the stage
completionDeferred = true
}
})
creator(Source.fromGraph(entitySource.source))
}
}
}

View file

@ -0,0 +1,208 @@
/*
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.impl.engine.server
import akka.http.impl.engine.parsing.ParserOutput
import akka.http.impl.engine.parsing.ParserOutput.{ StrictEntityCreator, EntityStreamError, EntityChunk, StreamedEntityCreator }
import akka.http.impl.engine.server.HttpServerBluePrint.PrepareRequests
import akka.http.scaladsl.model._
import akka.http.scaladsl.settings.ServerSettings
import akka.stream.{ Attributes, ActorMaterializer }
import akka.stream.scaladsl.{ Sink, Source, Flow }
import akka.stream.testkit.{ TestSubscriber, TestPublisher }
import akka.testkit.AkkaSpec
import akka.util.ByteString
import org.scalatest.{ Matchers, WordSpec }
import scala.concurrent.duration._
class PrepareRequestsSpec extends AkkaSpec {
val chunkedStart =
ParserOutput.RequestStart(
HttpMethods.GET,
Uri("http://example.com/"),
HttpProtocols.`HTTP/1.1`,
List(),
StreamedEntityCreator[ParserOutput, RequestEntity] { entityChunks
val chunks = entityChunks.collect {
case EntityChunk(chunk) chunk
case EntityStreamError(info) throw EntityStreamException(info)
}
HttpEntity.Chunked(ContentTypes.`application/octet-stream`, HttpEntity.limitableChunkSource(chunks))
},
true,
false)
val chunkPart =
ParserOutput.EntityChunk(HttpEntity.ChunkStreamPart(ByteString("abc")))
val chunkRequestComplete =
ParserOutput.MessageEnd
val strictRequest =
ParserOutput.RequestStart(
HttpMethods.GET,
Uri("http://example.com/"),
HttpProtocols.`HTTP/1.1`,
List(),
StrictEntityCreator(HttpEntity.Strict(ContentTypes.`application/octet-stream`, ByteString("body"))),
true,
false)
"The PrepareRequest stage" should {
"not fail when there is demand from both streamed entity consumption and regular flow" in {
implicit val materializer = ActorMaterializer()
// covers bug #19623 where a reply before the streamed
// body has been consumed causes pull/push twice
val inProbe = TestPublisher.manualProbe[ParserOutput.RequestOutput]()
val upstreamProbe = TestSubscriber.manualProbe[HttpRequest]()
val stage = Flow.fromGraph(new PrepareRequests(ServerSettings(system)))
Source.fromPublisher(inProbe)
.via(stage)
.to(Sink.fromSubscriber(upstreamProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val upstreamSub = upstreamProbe.expectSubscription()
val inSub = inProbe.expectSubscription()
// let request with streamed entity through
upstreamSub.request(1)
inSub.expectRequest(1)
inSub.sendNext(chunkedStart)
val request = upstreamProbe.expectNext()
// and subscribe to it's streamed entity
val entityProbe = TestSubscriber.manualProbe[ByteString]()
request.entity.dataBytes.to(Sink.fromSubscriber(entityProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val entitySub = entityProbe.expectSubscription()
// the bug happens when both the client has signalled demand
// and the the streamed entity has
upstreamSub.request(1)
entitySub.request(1)
// then comes the next chunk from the actual request
inSub.expectRequest(1)
// bug would fail stream here with exception
upstreamProbe.expectNoMsg(100.millis)
inSub.sendNext(ParserOutput.EntityChunk(HttpEntity.ChunkStreamPart(ByteString("abc"))))
entityProbe.expectNext()
entitySub.request(1)
inSub.sendNext(ParserOutput.MessageEnd)
entityProbe.expectComplete()
// the rest of the test covers the saved pull
// that should go downstream when the streamed entity
// has reached it's end
inSub.expectRequest(1)
inSub.sendNext(strictRequest)
upstreamProbe.expectNext()
}
"not complete running entity stream when upstream cancels" in {
implicit val materializer = ActorMaterializer()
val inProbe = TestPublisher.manualProbe[ParserOutput.RequestOutput]()
val upstreamProbe = TestSubscriber.manualProbe[HttpRequest]()
val stage = Flow.fromGraph(new PrepareRequests(ServerSettings(system)))
Source.fromPublisher(inProbe)
.via(stage)
.to(Sink.fromSubscriber(upstreamProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val upstreamSub = upstreamProbe.expectSubscription()
val inSub = inProbe.expectSubscription()
// let request with streamed entity through
upstreamSub.request(1)
inSub.expectRequest(1)
inSub.sendNext(chunkedStart)
val request = upstreamProbe.expectNext()
// and subscribe to it's streamed entity
val entityProbe = TestSubscriber.manualProbe[ByteString]()
request.entity.dataBytes.to(Sink.fromSubscriber(entityProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val entitySub = entityProbe.expectSubscription()
// user logic cancels flow
upstreamSub.cancel()
// but incoming chunks should still end up in entity
entitySub.request(1)
inSub.expectRequest(1)
inSub.sendNext(chunkPart)
entityProbe.expectNext()
entitySub.request(1)
inSub.expectRequest(1)
inSub.sendNext(chunkRequestComplete)
// and then when entity is complete, the stage should complete
entityProbe.expectComplete()
}
"complete stage if chunked stream is completed without reaching end of chunks" in {
// a bit unsure about this, but to document the assumption
implicit val materializer = ActorMaterializer()
val inProbe = TestPublisher.manualProbe[ParserOutput.RequestOutput]()
val upstreamProbe = TestSubscriber.manualProbe[HttpRequest]()
val stage = Flow.fromGraph(new PrepareRequests(ServerSettings(system)))
Source.fromPublisher(inProbe)
.via(stage)
.to(Sink.fromSubscriber(upstreamProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val upstreamSub = upstreamProbe.expectSubscription()
val inSub = inProbe.expectSubscription()
// let request with streamed entity through
upstreamSub.request(1)
inSub.expectRequest(1)
inSub.sendNext(chunkedStart)
val request = upstreamProbe.expectNext()
// and subscribe to it's streamed entity
val entityProbe = TestSubscriber.manualProbe[ByteString]()
request.entity.dataBytes.to(Sink.fromSubscriber(entityProbe))
.withAttributes(Attributes.inputBuffer(1, 1))
.run()
val entitySub = entityProbe.expectSubscription()
// incoming stream is completed, but we never got the chunk end
inSub.sendComplete()
// assumption: should cause stage to complete
entityProbe.expectComplete()
upstreamProbe.expectComplete()
}
}
}