Merge pull request #17105 from akka/wip-async-stage-∂π

add async stage
This commit is contained in:
Roland Kuhn 2015-04-10 10:52:01 +02:00
commit 8f47b6dfcc
66 changed files with 981 additions and 787 deletions

View file

@ -96,7 +96,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: FlowMaterializer): Future[ServerBinding] =
bindAndHandle(Flow[HttpRequest].mapAsync(handler), interface, port, backlog, options, settings, log)
bindAndHandle(Flow[HttpRequest].mapAsync(1, handler), interface, port, backlog, options, settings, log)
/**
* Transforms a given HTTP-level server [[Flow]] into a lower-level TCP transport flow.
@ -215,7 +215,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
* Returns the materialization result of the underlying flow materialization.
*/
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): Unit =
handleWith(Flow[HttpRequest].mapAsync(handler))
handleWith(Flow[HttpRequest].mapAsync(1, handler))
}
/**

View file

@ -253,8 +253,8 @@ private[http] object HttpClient {
def recover[A, B >: A](pf: PartialFunction[Throwable, B]): () PushPullStage[A, B] = {
val stage = new PushPullStage[A, B] {
var recovery: Option[B] = None
def onPush(elem: A, ctx: Context[B]): Directive = ctx.push(elem)
def onPull(ctx: Context[B]): Directive = recovery match {
def onPush(elem: A, ctx: Context[B]): SyncDirective = ctx.push(elem)
def onPull(ctx: Context[B]): SyncDirective = recovery match {
case None ctx.pull()
case Some(x) { recovery = null; ctx.push(x) }
case null ctx.finish()

View file

@ -61,7 +61,7 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
def warnOnIllegalHeader(errorInfo: ErrorInfo): Unit =
if (illegalHeaderWarnings) log.warning(errorInfo.withSummaryPrepended("Illegal multipart header").formatPretty)
override def onPush(input: ByteString, ctx: Context[Output]): Directive = {
override def onPush(input: ByteString, ctx: Context[Output]): SyncDirective = {
try state(input)
catch {
case e: ParsingException fail(e.info)
@ -74,7 +74,7 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
else ctx.finish()
}
override def onPull(ctx: Context[Output]): Directive = {
override def onPull(ctx: Context[Output]): SyncDirective = {
if (output.nonEmpty)
ctx.push(dequeue())
else if (ctx.isFinishing) {
@ -274,4 +274,3 @@ private[http] object BodyPartParser {
illegalHeaderWarnings = true,
headerValueCacheLimit = 8)
}

View file

@ -37,7 +37,7 @@ private[http] abstract class HttpMessageParser[Output >: MessageOutput <: Parser
new PushPullStage[ByteString, Output] {
def onPush(elem: ByteString, ctx: Context[Output]) = handleParserOutput(self.onPush(elem), ctx)
def onPull(ctx: Context[Output]) = handleParserOutput(self.onPull(), ctx)
private def handleParserOutput(output: Output, ctx: Context[Output]): Directive =
private def handleParserOutput(output: Output, ctx: Context[Output]): SyncDirective =
output match {
case StreamEnd ctx.finish()
case NeedMoreData ctx.pull()
@ -342,4 +342,4 @@ private[http] object HttpMessageParser {
() Some(ParserOutput.MessageStartError(StatusCodes.BadRequest, ErrorInfo("Illegal HTTP message start")))
val CompletionIsEntityStreamError: CompletionHandling =
() Some(ParserOutput.EntityStreamError(ErrorInfo("Entity stream truncation")))
}
}

View file

@ -28,7 +28,7 @@ private[http] object BodyPartRenderer {
new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart, Unit]] {
var firstBoundaryRendered = false
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart, Unit]]): SyncDirective = {
val r = new CustomCharsetByteStringRendering(nioCharset, partHeadersSizeHint)
def bodyPartChunks(data: Source[ByteString, Unit]): Source[ChunkStreamPart, Unit] = {
@ -51,7 +51,7 @@ private[http] object BodyPartRenderer {
ctx.push(completePartRendering())
}
override def onPull(ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
override def onPull(ctx: Context[Source[ChunkStreamPart, Unit]]): SyncDirective = {
val finishing = ctx.isFinishing
if (finishing && firstBoundaryRendered) {
val r = new ByteStringRendering(boundary.length + 4)

View file

@ -27,7 +27,7 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString, Unit]] {
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString, Unit]]): SyncDirective = {
val r = new ByteStringRendering(requestHeaderSizeHint)
import ctx.request._

View file

@ -58,7 +58,7 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
// need this for testing
private[http] def isComplete = close
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString, Unit]]): SyncDirective = {
val r = new ByteStringRendering(responseHeaderSizeHint)
import ctx.response._

View file

@ -55,7 +55,7 @@ private object RenderSupport {
var lastChunkSeen = false
override def initial = new State {
override def onPush(chunk: HttpEntity.ChunkStreamPart, ctx: Context[ByteString]): Directive = {
override def onPush(chunk: HttpEntity.ChunkStreamPart, ctx: Context[ByteString]): SyncDirective = {
if (chunk.isLastChunk)
lastChunkSeen = true
ctx.push(renderChunk(chunk))
@ -70,7 +70,7 @@ private object RenderSupport {
class CheckContentLengthTransformer(length: Long) extends PushStage[ByteString, ByteString] {
var sent = 0L
override def onPush(elem: ByteString, ctx: Context[ByteString]): Directive = {
override def onPush(elem: ByteString, ctx: Context[ByteString]): SyncDirective = {
sent += elem.length
if (sent > length)
throw InvalidContentLengthException(s"HTTP message had declared Content-Length $length but entity data stream amounts to more bytes")

View file

@ -6,6 +6,7 @@ package akka.http.util
import akka.stream.stage.{ Directive, Context, StatefulStage }
import akka.util.ByteString
import akka.stream.stage.SyncDirective
/**
* A helper class for writing parsers from ByteStrings.
@ -15,15 +16,15 @@ import akka.util.ByteString
* INTERNAL API
*/
private[akka] abstract class ByteStringParserStage[Out] extends StatefulStage[ByteString, Out] {
protected def onTruncation(ctx: Context[Out]): Directive
protected def onTruncation(ctx: Context[Out]): SyncDirective
/**
* Derive a stage from [[IntermediateState]] and then call `pull(ctx)` instead of
* `ctx.pull()` to have truncation errors reported.
*/
abstract class IntermediateState extends State {
override def onPull(ctx: Context[Out]): Directive = pull(ctx)
def pull(ctx: Context[Out]): Directive =
override def onPull(ctx: Context[Out]): SyncDirective = pull(ctx)
def pull(ctx: Context[Out]): SyncDirective =
if (ctx.isFinishing) onTruncation(ctx)
else ctx.pull()
}
@ -37,9 +38,9 @@ private[akka] abstract class ByteStringParserStage[Out] extends StatefulStage[By
* manipulate any state during reading from the ByteReader.
*/
trait ByteReadingState extends IntermediateState {
def read(reader: ByteReader, ctx: Context[Out]): Directive
def read(reader: ByteReader, ctx: Context[Out]): SyncDirective
def onPush(data: ByteString, ctx: Context[Out]): Directive =
def onPush(data: ByteString, ctx: Context[Out]): SyncDirective =
try {
val reader = new ByteReader(data)
read(reader, ctx)
@ -50,9 +51,9 @@ private[akka] abstract class ByteStringParserStage[Out] extends StatefulStage[By
}
}
case class TryAgain(previousData: ByteString, byteReadingState: ByteReadingState) extends IntermediateState {
def onPush(data: ByteString, ctx: Context[Out]): Directive = {
def onPush(data: ByteString, ctx: Context[Out]): SyncDirective = {
become(byteReadingState)
byteReadingState.onPush(previousData ++ data, ctx)
}
}
}
}

View file

@ -27,10 +27,10 @@ private[http] object StreamUtils {
*/
def byteStringTransformer(f: ByteString ByteString, finish: () ByteString): Stage[ByteString, ByteString] = {
new PushPullStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective =
ctx.push(f(element))
override def onPull(ctx: Context[ByteString]): Directive =
override def onPull(ctx: Context[ByteString]): SyncDirective =
if (ctx.isFinishing) ctx.pushAndFinish(finish())
else ctx.pull()
@ -43,7 +43,7 @@ private[http] object StreamUtils {
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString, Unit] = {
val transformer = new PushStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective =
ctx.push(element)
override def onUpstreamFailure(cause: Throwable, ctx: Context[ByteString]): TerminationDirective =
@ -59,7 +59,7 @@ private[http] object StreamUtils {
def skipping = new State {
var toSkip = start
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective =
if (element.length < toSkip) {
// keep skipping
toSkip -= element.length
@ -74,7 +74,7 @@ private[http] object StreamUtils {
def taking(initiallyRemaining: Long) = new State {
var remaining: Long = initiallyRemaining
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = {
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective = {
val data = element.take(math.min(remaining, Int.MaxValue).toInt)
remaining -= data.size
if (remaining <= 0) ctx.pushAndFinish(data)
@ -92,7 +92,7 @@ private[http] object StreamUtils {
def initial = WaitingForData
case object WaitingForData extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
def onPush(elem: ByteString, ctx: Context[ByteString]): SyncDirective =
if (elem.size <= maxBytesPerChunk) ctx.push(elem)
else {
become(DeliveringData(elem.drop(maxBytesPerChunk)))
@ -101,10 +101,10 @@ private[http] object StreamUtils {
}
case class DeliveringData(remaining: ByteString) extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
def onPush(elem: ByteString, ctx: Context[ByteString]): SyncDirective =
throw new IllegalStateException("Not expecting data")
override def onPull(ctx: Context[ByteString]): Directive = {
override def onPull(ctx: Context[ByteString]): SyncDirective = {
val toPush = remaining.take(maxBytesPerChunk)
val toKeep = remaining.drop(maxBytesPerChunk)

View file

@ -55,7 +55,7 @@ package object util {
def printEvent[T](marker: String): Flow[T, T, Unit] =
Flow[T].transform(() new PushStage[T, T] {
override def onPush(element: T, ctx: Context[T]): Directive = {
override def onPush(element: T, ctx: Context[T]): SyncDirective = {
println(s"$marker: $element")
ctx.push(element)
}
@ -86,7 +86,7 @@ package object util {
private[http] def errorLogger(log: LoggingAdapter, msg: String): PushStage[ByteString, ByteString] =
new PushStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = ctx.push(element)
override def onPush(element: ByteString, ctx: Context[ByteString]): SyncDirective = ctx.push(element)
override def onUpstreamFailure(cause: Throwable, ctx: Context[ByteString]): TerminationDirective = {
log.error(cause, msg)
super.onUpstreamFailure(cause, ctx)
@ -106,4 +106,3 @@ package object util {
} else bytes.toString + " B"
}
}