!str #15236 Replace Transformer with Stage
* replace all existing Transformer with Stage (PushPullStage) * use Flow[ByteString, ByteString] as encoder/decoder transformer in http * use the IteratorInterpreter for strict if possible * emit then become * emit then finish * termination emits * FlowTransformerSpec * rework types to work with Java API * rename and move things * add scaladoc
This commit is contained in:
parent
299854905b
commit
a82f266367
61 changed files with 1478 additions and 1518 deletions
|
|
@ -8,8 +8,8 @@ import scala.annotation.tailrec
|
|||
import scala.collection.mutable.ListBuffer
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.parboiled2.CharPredicate
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
import akka.http.model._
|
||||
import akka.http.util._
|
||||
|
|
@ -24,7 +24,7 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
|||
boundary: String,
|
||||
log: LoggingAdapter,
|
||||
settings: BodyPartParser.Settings = BodyPartParser.defaultSettings)
|
||||
extends Transformer[ByteString, BodyPartParser.Output] {
|
||||
extends PushPullStage[ByteString, BodyPartParser.Output] {
|
||||
import BodyPartParser._
|
||||
import settings._
|
||||
|
||||
|
|
@ -52,25 +52,43 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
|||
|
||||
private[this] val headerParser = HttpHeaderParser(settings, warnOnIllegalHeader) // TODO: prevent re-priming header parser from scratch
|
||||
private[this] val result = new ListBuffer[Output] // transformer op is currently optimized for LinearSeqs
|
||||
private[this] var resultIterator: Iterator[Output] = Iterator.empty
|
||||
private[this] var state: ByteString ⇒ StateResult = tryParseInitialBoundary
|
||||
private[this] var receivedInitialBoundary = false
|
||||
private[this] var terminated = false
|
||||
|
||||
override def isComplete = terminated
|
||||
|
||||
def warnOnIllegalHeader(errorInfo: ErrorInfo): Unit =
|
||||
if (illegalHeaderWarnings) log.warning(errorInfo.withSummaryPrepended("Illegal multipart header").formatPretty)
|
||||
|
||||
def onNext(input: ByteString): List[Output] = {
|
||||
override def onPush(input: ByteString, ctx: Context[Output]): Directive = {
|
||||
result.clear()
|
||||
try state(input)
|
||||
catch {
|
||||
case e: ParsingException ⇒ fail(e.info)
|
||||
case NotEnoughDataException ⇒ throw new IllegalStateException(NotEnoughDataException) // we are missing a try/catch{continue} wrapper somewhere
|
||||
case NotEnoughDataException ⇒
|
||||
// we are missing a try/catch{continue} wrapper somewhere
|
||||
throw new IllegalStateException("unexpected NotEnoughDataException", NotEnoughDataException)
|
||||
}
|
||||
result.toList
|
||||
resultIterator = result.iterator
|
||||
if (resultIterator.hasNext) ctx.push(resultIterator.next())
|
||||
else if (!terminated) ctx.pull()
|
||||
else ctx.finish()
|
||||
}
|
||||
|
||||
override def onPull(ctx: Context[Output]): Directive = {
|
||||
if (resultIterator.hasNext)
|
||||
ctx.push(resultIterator.next())
|
||||
else if (ctx.isFinishing) {
|
||||
if (terminated || !receivedInitialBoundary)
|
||||
ctx.finish()
|
||||
else
|
||||
ctx.pushAndFinish(ParseError(ErrorInfo("Unexpected end of multipart entity")))
|
||||
} else
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Output]): TerminationDirective = ctx.absorbTermination()
|
||||
|
||||
def tryParseInitialBoundary(input: ByteString): StateResult =
|
||||
// we don't use boyerMoore here because we are testing for the boundary *without* a
|
||||
// preceding CRLF and at a known location (the very beginning of the entity)
|
||||
|
|
@ -223,8 +241,6 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
|||
def doubleDash(input: ByteString, offset: Int): Boolean =
|
||||
byteChar(input, offset) == '-' && byteChar(input, offset + 1) == '-'
|
||||
|
||||
override def onTermination(e: Option[Throwable]): List[BodyPartParser.Output] =
|
||||
if (terminated || !receivedInitialBoundary) Nil else ParseError(ErrorInfo("Unexpected end of multipart entity")) :: Nil
|
||||
}
|
||||
|
||||
private[http] object BodyPartParser {
|
||||
|
|
|
|||
|
|
@ -9,8 +9,8 @@ import scala.collection.mutable.ListBuffer
|
|||
import scala.collection.immutable
|
||||
import akka.parboiled2.CharUtils
|
||||
import akka.util.ByteString
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.stage._
|
||||
import akka.http.model.parser.CharacterClasses
|
||||
import akka.http.model._
|
||||
import headers._
|
||||
|
|
@ -21,7 +21,7 @@ import HttpProtocols._
|
|||
*/
|
||||
private[http] abstract class HttpMessageParser[Output >: ParserOutput.MessageOutput <: ParserOutput](val settings: ParserSettings,
|
||||
val headerParser: HttpHeaderParser)
|
||||
extends Transformer[ByteString, Output] {
|
||||
extends StatefulStage[ByteString, Output] {
|
||||
import settings._
|
||||
|
||||
sealed trait StateResult // phantom type for ensuring soundness of our parsing method setup
|
||||
|
|
@ -30,16 +30,21 @@ private[http] abstract class HttpMessageParser[Output >: ParserOutput.MessageOut
|
|||
private[this] var state: ByteString ⇒ StateResult = startNewMessage(_, 0)
|
||||
private[this] var protocol: HttpProtocol = `HTTP/1.1`
|
||||
private[this] var terminated = false
|
||||
override def isComplete = terminated
|
||||
|
||||
def onNext(input: ByteString): immutable.Seq[Output] = {
|
||||
override def initial = new State {
|
||||
override def onPush(input: ByteString, ctx: Context[Output]): Directive = {
|
||||
result.clear()
|
||||
try state(input)
|
||||
catch {
|
||||
case e: ParsingException ⇒ fail(e.status, e.info)
|
||||
case NotEnoughDataException ⇒ throw new IllegalStateException // we are missing a try/catch{continue} wrapper somewhere
|
||||
case NotEnoughDataException ⇒
|
||||
// we are missing a try/catch{continue} wrapper somewhere
|
||||
throw new IllegalStateException("unexpected NotEnoughDataException", NotEnoughDataException)
|
||||
}
|
||||
val resultIterator = result.iterator
|
||||
if (terminated) emitAndFinish(resultIterator, ctx)
|
||||
else emit(resultIterator, ctx)
|
||||
}
|
||||
result.toList
|
||||
}
|
||||
|
||||
def startNewMessage(input: ByteString, offset: Int): StateResult = {
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import akka.http.model.headers._
|
|||
import akka.http.engine.rendering.RenderSupport._
|
||||
import akka.http.util._
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
import HttpEntity._
|
||||
|
||||
|
|
@ -24,19 +24,19 @@ private[http] object BodyPartRenderer {
|
|||
def streamed(boundary: String,
|
||||
nioCharset: Charset,
|
||||
partHeadersSizeHint: Int,
|
||||
log: LoggingAdapter): Transformer[Multipart.BodyPart, Source[ChunkStreamPart]] =
|
||||
new Transformer[Multipart.BodyPart, Source[ChunkStreamPart]] {
|
||||
log: LoggingAdapter): PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] =
|
||||
new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] {
|
||||
var firstBoundaryRendered = false
|
||||
|
||||
def onNext(bodyPart: Multipart.BodyPart): List[Source[ChunkStreamPart]] = {
|
||||
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart]]): Directive = {
|
||||
val r = new CustomCharsetByteStringRendering(nioCharset, partHeadersSizeHint)
|
||||
|
||||
def bodyPartChunks(data: Source[ByteString]): List[Source[ChunkStreamPart]] = {
|
||||
def bodyPartChunks(data: Source[ByteString]): Source[ChunkStreamPart] = {
|
||||
val entityChunks = data.map[ChunkStreamPart](Chunk(_))
|
||||
(Source(Chunk(r.get) :: Nil) ++ entityChunks) :: Nil
|
||||
chunkStream(r.get) ++ entityChunks
|
||||
}
|
||||
|
||||
def completePartRendering(): List[Source[ChunkStreamPart]] =
|
||||
def completePartRendering(): Source[ChunkStreamPart] =
|
||||
bodyPart.entity match {
|
||||
case x if x.isKnownEmpty ⇒ chunkStream(r.get)
|
||||
case Strict(_, data) ⇒ chunkStream((r ~~ data).get)
|
||||
|
|
@ -48,18 +48,26 @@ private[http] object BodyPartRenderer {
|
|||
firstBoundaryRendered = true
|
||||
renderEntityContentType(r, bodyPart.entity)
|
||||
renderHeaders(r, bodyPart.headers, log)
|
||||
completePartRendering()
|
||||
ctx.push(completePartRendering())
|
||||
}
|
||||
|
||||
override def onTermination(e: Option[Throwable]): List[Source[ChunkStreamPart]] =
|
||||
if (e.isEmpty && firstBoundaryRendered) {
|
||||
override def onPull(ctx: Context[Source[ChunkStreamPart]]): Directive = {
|
||||
val finishing = ctx.isFinishing
|
||||
if (finishing && firstBoundaryRendered) {
|
||||
val r = new ByteStringRendering(boundary.length + 4)
|
||||
renderFinalBoundary(r, boundary)
|
||||
chunkStream(r.get)
|
||||
} else Nil
|
||||
ctx.pushAndFinish(chunkStream(r.get))
|
||||
} else if (finishing)
|
||||
ctx.finish()
|
||||
else
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Source[ChunkStreamPart]]): TerminationDirective = ctx.absorbTermination()
|
||||
|
||||
private def chunkStream(byteString: ByteString): Source[ChunkStreamPart] =
|
||||
Source.singleton(Chunk(byteString))
|
||||
|
||||
private def chunkStream(byteString: ByteString) =
|
||||
Source[ChunkStreamPart](Chunk(byteString) :: Nil) :: Nil
|
||||
}
|
||||
|
||||
def strict(parts: immutable.Seq[Multipart.BodyPart.Strict], boundary: String, nioCharset: Charset,
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import scala.annotation.tailrec
|
|||
import akka.event.LoggingAdapter
|
||||
import akka.util.ByteString
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.stage._
|
||||
import akka.http.model._
|
||||
import akka.http.util._
|
||||
import RenderSupport._
|
||||
|
|
@ -24,9 +24,9 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
|
|||
|
||||
def newRenderer: HttpRequestRenderer = new HttpRequestRenderer
|
||||
|
||||
final class HttpRequestRenderer extends Transformer[RequestRenderingContext, Source[ByteString]] {
|
||||
final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString]] {
|
||||
|
||||
def onNext(ctx: RequestRenderingContext): List[Source[ByteString]] = {
|
||||
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString]]): Directive = {
|
||||
val r = new ByteStringRendering(requestHeaderSizeHint)
|
||||
import ctx.request._
|
||||
|
||||
|
|
@ -103,15 +103,15 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
|
|||
r ~~ CrLf
|
||||
}
|
||||
|
||||
def completeRequestRendering(): List[Source[ByteString]] =
|
||||
def completeRequestRendering(): Source[ByteString] =
|
||||
entity match {
|
||||
case x if x.isKnownEmpty ⇒
|
||||
renderContentLength(0)
|
||||
Source(r.get :: Nil) :: Nil
|
||||
Source.singleton(r.get)
|
||||
|
||||
case HttpEntity.Strict(_, data) ⇒
|
||||
renderContentLength(data.length)
|
||||
Source.singleton(r.get ++ data) :: Nil
|
||||
Source.singleton(r.get ++ data)
|
||||
|
||||
case HttpEntity.Default(_, contentLength, data) ⇒
|
||||
renderContentLength(contentLength)
|
||||
|
|
@ -126,7 +126,7 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
|
|||
renderRequestLine()
|
||||
renderHeaders(headers.toList)
|
||||
renderEntityContentType(r, entity)
|
||||
completeRequestRendering()
|
||||
opCtx.push(completeRequestRendering())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.annotation.tailrec
|
|||
import akka.event.LoggingAdapter
|
||||
import akka.util.ByteString
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.stage._
|
||||
import akka.http.model._
|
||||
import akka.http.util._
|
||||
import RenderSupport._
|
||||
|
|
@ -50,12 +50,14 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
|
|||
|
||||
def newRenderer: HttpResponseRenderer = new HttpResponseRenderer
|
||||
|
||||
final class HttpResponseRenderer extends Transformer[ResponseRenderingContext, Source[ByteString]] {
|
||||
final class HttpResponseRenderer extends PushStage[ResponseRenderingContext, Source[ByteString]] {
|
||||
|
||||
private[this] var close = false // signals whether the connection is to be closed after the current response
|
||||
|
||||
override def isComplete = close
|
||||
// need this for testing
|
||||
private[http] def isComplete = close
|
||||
|
||||
def onNext(ctx: ResponseRenderingContext): List[Source[ByteString]] = {
|
||||
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString]]): Directive = {
|
||||
val r = new ByteStringRendering(responseHeaderSizeHint)
|
||||
|
||||
import ctx.response._
|
||||
|
|
@ -134,17 +136,17 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
|
|||
r ~~ `Transfer-Encoding` ~~ ChunkedBytes ~~ CrLf
|
||||
}
|
||||
|
||||
def byteStrings(entityBytes: ⇒ Source[ByteString]): List[Source[ByteString]] =
|
||||
def byteStrings(entityBytes: ⇒ Source[ByteString]): Source[ByteString] =
|
||||
renderByteStrings(r, entityBytes, skipEntity = noEntity)
|
||||
|
||||
def completeResponseRendering(entity: ResponseEntity): List[Source[ByteString]] =
|
||||
def completeResponseRendering(entity: ResponseEntity): Source[ByteString] =
|
||||
entity match {
|
||||
case HttpEntity.Strict(_, data) ⇒
|
||||
renderHeaders(headers.toList)
|
||||
renderEntityContentType(r, entity)
|
||||
r ~~ `Content-Length` ~~ data.length ~~ CrLf ~~ CrLf
|
||||
val entityBytes = if (noEntity) ByteString.empty else data
|
||||
Source.singleton(r.get ++ entityBytes) :: Nil
|
||||
Source.singleton(r.get ++ entityBytes)
|
||||
|
||||
case HttpEntity.Default(_, contentLength, data) ⇒
|
||||
renderHeaders(headers.toList)
|
||||
|
|
@ -170,7 +172,11 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
|
|||
}
|
||||
|
||||
renderStatusLine()
|
||||
completeResponseRendering(entity)
|
||||
val result = completeResponseRendering(entity)
|
||||
if (close)
|
||||
opCtx.pushAndFinish(result)
|
||||
else
|
||||
opCtx.push(result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.stream.impl.ActorBasedFlowMaterializer
|
|||
import akka.util.ByteString
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.stage._
|
||||
import akka.http.model._
|
||||
import akka.http.util._
|
||||
import org.reactivestreams.Subscriber
|
||||
|
|
@ -45,38 +45,46 @@ private object RenderSupport {
|
|||
r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf
|
||||
|
||||
def renderByteStrings(r: ByteStringRendering, entityBytes: ⇒ Source[ByteString],
|
||||
skipEntity: Boolean = false): List[Source[ByteString]] = {
|
||||
val messageStart = Source(r.get :: Nil)
|
||||
skipEntity: Boolean = false): Source[ByteString] = {
|
||||
val messageStart = Source.singleton(r.get)
|
||||
val messageBytes =
|
||||
if (!skipEntity) messageStart ++ entityBytes
|
||||
else CancelSecond(messageStart, entityBytes)
|
||||
messageBytes :: Nil
|
||||
messageBytes
|
||||
}
|
||||
|
||||
class ChunkTransformer extends Transformer[HttpEntity.ChunkStreamPart, ByteString] {
|
||||
class ChunkTransformer extends StatefulStage[HttpEntity.ChunkStreamPart, ByteString] {
|
||||
var lastChunkSeen = false
|
||||
def onNext(chunk: HttpEntity.ChunkStreamPart): List[ByteString] = {
|
||||
if (chunk.isLastChunk) lastChunkSeen = true
|
||||
renderChunk(chunk) :: Nil
|
||||
|
||||
override def initial = new State {
|
||||
override def onPush(chunk: HttpEntity.ChunkStreamPart, ctx: Context[ByteString]): Directive = {
|
||||
if (chunk.isLastChunk)
|
||||
lastChunkSeen = true
|
||||
ctx.push(renderChunk(chunk))
|
||||
}
|
||||
override def isComplete = lastChunkSeen
|
||||
override def onTermination(e: Option[Throwable]) = if (lastChunkSeen) Nil else defaultLastChunkBytes :: Nil
|
||||
}
|
||||
|
||||
class CheckContentLengthTransformer(length: Long) extends Transformer[ByteString, ByteString] {
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective =
|
||||
if (lastChunkSeen) super.onUpstreamFinish(ctx)
|
||||
else terminationEmit(Iterator.single(defaultLastChunkBytes), ctx)
|
||||
}
|
||||
|
||||
class CheckContentLengthTransformer(length: Long) extends PushStage[ByteString, ByteString] {
|
||||
var sent = 0L
|
||||
def onNext(elem: ByteString): List[ByteString] = {
|
||||
|
||||
override def onPush(elem: ByteString, ctx: Context[ByteString]): Directive = {
|
||||
sent += elem.length
|
||||
if (sent > length)
|
||||
throw new InvalidContentLengthException(s"HTTP message had declared Content-Length $length but entity data stream amounts to more bytes")
|
||||
elem :: Nil
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onTermination(e: Option[Throwable]): List[ByteString] = {
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective = {
|
||||
if (sent < length)
|
||||
throw new InvalidContentLengthException(s"HTTP message had declared Content-Length $length but entity data stream amounts to ${length - sent} bytes less")
|
||||
Nil
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private def renderChunk(chunk: HttpEntity.ChunkStreamPart): ByteString = {
|
||||
|
|
|
|||
|
|
@ -8,8 +8,8 @@ import akka.event.LoggingAdapter
|
|||
import akka.stream.io.StreamTcp
|
||||
import akka.stream.FlattenStrategy
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.http.engine.parsing.HttpRequestParser
|
||||
import akka.http.engine.rendering.{ ResponseRenderingContext, HttpResponseRendererFactory }
|
||||
import akka.http.model.{ StatusCode, ErrorInfo, HttpRequest, HttpResponse, HttpMethods }
|
||||
|
|
@ -84,37 +84,38 @@ private[http] class HttpServerPipeline(settings: ServerSettings, log: LoggingAda
|
|||
* If the parser produced a ParserOutput.ParseError the error response is immediately dispatched to downstream.
|
||||
*/
|
||||
def applyApplicationBypass =
|
||||
new Transformer[Any, ResponseRenderingContext] {
|
||||
new PushStage[Any, ResponseRenderingContext] {
|
||||
var applicationResponse: HttpResponse = _
|
||||
var requestStart: RequestStart = _
|
||||
|
||||
def onNext(elem: Any) = elem match {
|
||||
override def onPush(elem: Any, ctx: Context[ResponseRenderingContext]): Directive = elem match {
|
||||
case response: HttpResponse ⇒
|
||||
requestStart match {
|
||||
case null ⇒
|
||||
applicationResponse = response
|
||||
Nil
|
||||
ctx.pull()
|
||||
case x: RequestStart ⇒
|
||||
requestStart = null
|
||||
dispatch(x, response)
|
||||
ctx.push(dispatch(x, response))
|
||||
}
|
||||
|
||||
case requestStart: RequestStart ⇒
|
||||
applicationResponse match {
|
||||
case null ⇒
|
||||
this.requestStart = requestStart
|
||||
Nil
|
||||
ctx.pull()
|
||||
case response ⇒
|
||||
applicationResponse = null
|
||||
dispatch(requestStart, response)
|
||||
ctx.push(dispatch(requestStart, response))
|
||||
}
|
||||
|
||||
case ParseError(status, info) ⇒ errorResponse(status, info) :: Nil
|
||||
case ParseError(status, info) ⇒
|
||||
ctx.push(errorResponse(status, info))
|
||||
}
|
||||
|
||||
def dispatch(requestStart: RequestStart, response: HttpResponse): List[ResponseRenderingContext] = {
|
||||
def dispatch(requestStart: RequestStart, response: HttpResponse): ResponseRenderingContext = {
|
||||
import requestStart._
|
||||
ResponseRenderingContext(response, method, protocol, closeAfterResponseCompletion) :: Nil
|
||||
ResponseRenderingContext(response, method, protocol, closeAfterResponseCompletion)
|
||||
}
|
||||
|
||||
def errorResponse(status: StatusCode, info: ErrorInfo): ResponseRenderingContext = {
|
||||
|
|
|
|||
|
|
@ -14,9 +14,11 @@ import scala.util.control.NonFatal
|
|||
import akka.util.ByteString
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.{ TimerTransformer, Transformer }
|
||||
import akka.stream.TimerTransformer
|
||||
import akka.http.util._
|
||||
import japi.JavaMapping.Implicits._
|
||||
import scala.util.Success
|
||||
import scala.util.Failure
|
||||
|
||||
/**
|
||||
* Models the entity (aka "body" or "content) of an HTTP message.
|
||||
|
|
@ -70,7 +72,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
|
|||
* This method may only throw an exception if the `transformer` function throws an exception while creating the transformer.
|
||||
* Any other errors are reported through the new entity data stream.
|
||||
*/
|
||||
def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): HttpEntity
|
||||
def transformDataBytes(transformer: Flow[ByteString, ByteString]): HttpEntity
|
||||
|
||||
/**
|
||||
* Creates a copy of this HttpEntity with the `contentType` overridden with the given one.
|
||||
|
|
@ -95,13 +97,13 @@ sealed trait BodyPartEntity extends HttpEntity with japi.BodyPartEntity {
|
|||
sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity {
|
||||
def withContentType(contentType: ContentType): RequestEntity
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): RequestEntity
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): RequestEntity
|
||||
}
|
||||
/* An entity that can be used for responses */
|
||||
sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity {
|
||||
def withContentType(contentType: ContentType): ResponseEntity
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): ResponseEntity
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): ResponseEntity
|
||||
}
|
||||
/* An entity that can be used for requests, responses, and body parts */
|
||||
sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity {
|
||||
|
|
@ -112,7 +114,7 @@ sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity wit
|
|||
* Transforms this' entities data bytes with a transformer that will produce exactly the number of bytes given as
|
||||
* ``newContentLength``.
|
||||
*/
|
||||
def transformDataBytes(newContentLength: Long, transformer: () ⇒ Transformer[ByteString, ByteString]): UniversalEntity
|
||||
def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity
|
||||
}
|
||||
|
||||
object HttpEntity {
|
||||
|
|
@ -143,11 +145,20 @@ object HttpEntity {
|
|||
// TODO: re-establish serializability
|
||||
// TODO: equal/hashcode ?
|
||||
|
||||
object Strict {
|
||||
// FIXME configurable?
|
||||
private val MaxByteSize = 1L * 1024 * 1024 * 1024
|
||||
private val MaxElements = 1000
|
||||
}
|
||||
|
||||
/**
|
||||
* The model for the entity of a "regular" unchunked HTTP message with known, fixed data.
|
||||
*/
|
||||
final case class Strict(contentType: ContentType, data: ByteString)
|
||||
extends japi.HttpEntityStrict with UniversalEntity {
|
||||
|
||||
import Strict._
|
||||
|
||||
def contentLength: Long = data.length
|
||||
|
||||
def isKnownEmpty: Boolean = data.isEmpty
|
||||
|
|
@ -157,21 +168,26 @@ object HttpEntity {
|
|||
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
|
||||
FastFuture.successful(this)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): MessageEntity = {
|
||||
try {
|
||||
val t = transformer()
|
||||
val newData = (t.onNext(data) ++ t.onTermination(None)).join
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): MessageEntity =
|
||||
StreamUtils.runStrict(data, transformer, MaxByteSize, MaxElements) match {
|
||||
case Success(Some(newData)) ⇒
|
||||
copy(data = newData)
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
case Success(None) ⇒
|
||||
Chunked.fromData(contentType, Source.singleton(data).via(transformer))
|
||||
case Failure(ex) ⇒
|
||||
Chunked(contentType, Source.failed(ex))
|
||||
}
|
||||
}
|
||||
override def transformDataBytes(newContentLength: Long, transformer: () ⇒ Transformer[ByteString, ByteString]): UniversalEntity = {
|
||||
val t = transformer()
|
||||
val newData = (t.onNext(data) ++ t.onTermination(None)).join
|
||||
assert(newData.length.toLong == newContentLength, s"Transformer didn't produce as much bytes (${newData.length}:'${newData.utf8String}') as claimed ($newContentLength)")
|
||||
|
||||
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity =
|
||||
StreamUtils.runStrict(data, transformer, MaxByteSize, MaxElements) match {
|
||||
case Success(Some(newData)) ⇒
|
||||
if (newData.length.toLong != newContentLength)
|
||||
throw new IllegalStateException(s"Transformer didn't produce as much bytes (${newData.length}:'${newData.utf8String}') as claimed ($newContentLength)")
|
||||
copy(data = newData)
|
||||
case Success(None) ⇒
|
||||
Default(contentType, newContentLength, Source.singleton(data).via(transformer))
|
||||
case Failure(ex) ⇒
|
||||
Default(contentType, newContentLength, Source.failed(ex))
|
||||
}
|
||||
|
||||
def withContentType(contentType: ContentType): Strict =
|
||||
|
|
@ -194,13 +210,11 @@ object HttpEntity {
|
|||
|
||||
def dataBytes: Source[ByteString] = data
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): Chunked = {
|
||||
val chunks = data.transform("transformDataBytes-Default", () ⇒ transformer().map(Chunk(_): ChunkStreamPart))
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked =
|
||||
Chunked.fromData(contentType, data.via(transformer))
|
||||
|
||||
HttpEntity.Chunked(contentType, chunks)
|
||||
}
|
||||
override def transformDataBytes(newContentLength: Long, transformer: () ⇒ Transformer[ByteString, ByteString]): UniversalEntity =
|
||||
Default(contentType, newContentLength, data.transform("transformDataBytes-with-new-length-Default", transformer))
|
||||
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity =
|
||||
Default(contentType, newContentLength, data.via(transformer))
|
||||
|
||||
def withContentType(contentType: ContentType): Default =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
|
@ -235,9 +249,8 @@ object HttpEntity {
|
|||
def withContentType(contentType: ContentType): CloseDelimited =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): CloseDelimited =
|
||||
HttpEntity.CloseDelimited(contentType,
|
||||
data.transform("transformDataBytes-CloseDelimited", transformer))
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): CloseDelimited =
|
||||
HttpEntity.CloseDelimited(contentType, data.via(transformer))
|
||||
|
||||
override def productPrefix = "HttpEntity.CloseDelimited"
|
||||
}
|
||||
|
|
@ -253,9 +266,8 @@ object HttpEntity {
|
|||
def withContentType(contentType: ContentType): IndefiniteLength =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): IndefiniteLength =
|
||||
HttpEntity.IndefiniteLength(contentType,
|
||||
data.transform("transformDataBytes-IndefiniteLength", transformer))
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): IndefiniteLength =
|
||||
HttpEntity.IndefiniteLength(contentType, data.via(transformer))
|
||||
|
||||
override def productPrefix = "HttpEntity.IndefiniteLength"
|
||||
}
|
||||
|
|
@ -272,35 +284,16 @@ object HttpEntity {
|
|||
def dataBytes: Source[ByteString] =
|
||||
chunks.map(_.data).filter(_.nonEmpty)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString]): Chunked = {
|
||||
val newChunks =
|
||||
chunks.transform("transformDataBytes-Chunked", () ⇒ new Transformer[ChunkStreamPart, ChunkStreamPart] {
|
||||
val byteTransformer = transformer()
|
||||
var sentLastChunk = false
|
||||
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked = {
|
||||
val newData =
|
||||
chunks.map {
|
||||
case Chunk(data, "") ⇒ data
|
||||
case LastChunk("", Nil) ⇒ ByteString.empty
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("Chunked.transformDataBytes not allowed for chunks with metadata")
|
||||
}.via(transformer)
|
||||
|
||||
override def isComplete: Boolean = byteTransformer.isComplete
|
||||
|
||||
def onNext(element: ChunkStreamPart): immutable.Seq[ChunkStreamPart] = element match {
|
||||
case Chunk(data, ext) ⇒ Chunk(byteTransformer.onNext(data).join, ext) :: Nil
|
||||
case l: LastChunk ⇒
|
||||
sentLastChunk = true
|
||||
Chunk(byteTransformer.onTermination(None).join) :: l :: Nil
|
||||
}
|
||||
override def onError(cause: scala.Throwable): Unit = byteTransformer.onError(cause)
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ChunkStreamPart] = {
|
||||
val remaining =
|
||||
if (e.isEmpty && !sentLastChunk) byteTransformer.onTermination(None)
|
||||
else if (e.isDefined /* && sentLastChunk */ ) byteTransformer.onTermination(e)
|
||||
else Nil
|
||||
|
||||
if (remaining.nonEmpty) Chunk(remaining.join) :: Nil
|
||||
else Nil
|
||||
}
|
||||
|
||||
override def cleanup(): Unit = byteTransformer.cleanup()
|
||||
})
|
||||
|
||||
HttpEntity.Chunked(contentType, newChunks)
|
||||
Chunked.fromData(contentType, newData)
|
||||
}
|
||||
|
||||
def withContentType(contentType: ContentType): Chunked =
|
||||
|
|
|
|||
|
|
@ -4,129 +4,120 @@
|
|||
|
||||
package akka.http.util
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.io.InputStream
|
||||
|
||||
import org.reactivestreams.{ Subscriber, Publisher }
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ ExecutionContext, Future }
|
||||
|
||||
import scala.util.Try
|
||||
import akka.actor.Props
|
||||
import akka.util.ByteString
|
||||
|
||||
import akka.stream.{ impl, Transformer, FlowMaterializer }
|
||||
import akka.stream.scaladsl._
|
||||
|
||||
import akka.http.model.RequestEntity
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.impl.Ast.AstNode
|
||||
import akka.stream.impl.Ast.StageFactory
|
||||
import akka.stream.impl.fusing.IteratorInterpreter
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.stream.impl
|
||||
import akka.util.ByteString
|
||||
import org.reactivestreams.{ Subscriber, Publisher }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] object StreamUtils {
|
||||
/**
|
||||
* Maps a transformer by strictly applying the given function to each output element.
|
||||
*/
|
||||
def mapTransformer[T, U, V](t: Transformer[T, U], f: U ⇒ V): Transformer[T, V] =
|
||||
new Transformer[T, V] {
|
||||
override def isComplete: Boolean = t.isComplete
|
||||
|
||||
def onNext(element: T): immutable.Seq[V] = t.onNext(element).map(f)
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[V] = t.onTermination(e).map(f)
|
||||
override def onError(cause: Throwable): Unit = t.onError(cause)
|
||||
override def cleanup(): Unit = t.cleanup()
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a transformer that will call `f` for each incoming ByteString and output its result. After the complete
|
||||
* input has been read it will call `finish` once to determine the final ByteString to post to the output.
|
||||
*/
|
||||
def byteStringTransformer(f: ByteString ⇒ ByteString, finish: () ⇒ ByteString): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] = f(element) :: Nil
|
||||
def byteStringTransformer(f: ByteString ⇒ ByteString, finish: () ⇒ ByteString): Flow[ByteString, ByteString] = {
|
||||
val transformer = new PushPullStage[ByteString, ByteString] {
|
||||
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
|
||||
ctx.push(f(element))
|
||||
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ByteString] =
|
||||
if (e.isEmpty) {
|
||||
val last = finish()
|
||||
if (last.nonEmpty) last :: Nil
|
||||
else Nil
|
||||
} else super.onTermination(e)
|
||||
override def onPull(ctx: Context[ByteString]): Directive =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(finish())
|
||||
else ctx.pull()
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
Flow[ByteString].transform("transformBytes", () ⇒ transformer)
|
||||
}
|
||||
|
||||
def failedPublisher[T](ex: Throwable): Publisher[T] =
|
||||
impl.ErrorPublisher(ex, "failed").asInstanceOf[Publisher[T]]
|
||||
|
||||
def mapErrorTransformer[T](f: Throwable ⇒ Throwable): Transformer[T, T] =
|
||||
new Transformer[T, T] {
|
||||
def onNext(element: T): immutable.Seq[T] = immutable.Seq(element)
|
||||
override def onError(cause: scala.Throwable): Unit = throw f(cause)
|
||||
def mapErrorTransformer(f: Throwable ⇒ Throwable): Flow[ByteString, ByteString] = {
|
||||
val transformer = new PushStage[ByteString, ByteString] {
|
||||
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
|
||||
ctx.push(element)
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[ByteString]): TerminationDirective =
|
||||
ctx.fail(f(cause))
|
||||
}
|
||||
|
||||
def sliceBytesTransformer(start: Long, length: Long): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
type State = Transformer[ByteString, ByteString]
|
||||
Flow[ByteString].transform("transformError", () ⇒ transformer)
|
||||
}
|
||||
|
||||
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString] = {
|
||||
val transformer = new StatefulStage[ByteString, ByteString] {
|
||||
|
||||
def skipping = new State {
|
||||
var toSkip = start
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] =
|
||||
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
|
||||
if (element.length < toSkip) {
|
||||
// keep skipping
|
||||
toSkip -= element.length
|
||||
Nil
|
||||
ctx.pull()
|
||||
} else {
|
||||
become(taking(length))
|
||||
// toSkip <= element.length <= Int.MaxValue
|
||||
currentState.onNext(element.drop(toSkip.toInt))
|
||||
current.onPush(element.drop(toSkip.toInt), ctx)
|
||||
}
|
||||
}
|
||||
def taking(initiallyRemaining: Long) = new State {
|
||||
var remaining: Long = initiallyRemaining
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] = {
|
||||
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = {
|
||||
val data = element.take(math.min(remaining, Int.MaxValue).toInt)
|
||||
remaining -= data.size
|
||||
if (remaining <= 0) become(finishing)
|
||||
data :: Nil
|
||||
if (remaining <= 0) ctx.pushAndFinish(data)
|
||||
else ctx.push(data)
|
||||
}
|
||||
}
|
||||
def finishing = new State {
|
||||
override def isComplete: Boolean = true
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] =
|
||||
throw new IllegalStateException("onNext called on complete stream")
|
||||
}
|
||||
|
||||
var currentState: State = if (start > 0) skipping else taking(length)
|
||||
def become(state: State): Unit = currentState = state
|
||||
|
||||
override def isComplete: Boolean = currentState.isComplete
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] = currentState.onNext(element)
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ByteString] = currentState.onTermination(e)
|
||||
override def initial: State = if (start > 0) skipping else taking(length)
|
||||
}
|
||||
Flow[ByteString].transform("sliceBytes", () ⇒ transformer)
|
||||
}
|
||||
|
||||
/**
|
||||
* 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[T, U](input: Source[T], transformers: immutable.Seq[() ⇒ Transformer[T, U]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[U]] =
|
||||
def transformMultiple(input: Source[ByteString], transformers: immutable.Seq[Flow[ByteString, ByteString]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString]] =
|
||||
transformers match {
|
||||
case Nil ⇒ Nil
|
||||
case Seq(one) ⇒ Vector(input.transform("transformMultipleElement", one))
|
||||
case Seq(one) ⇒ Vector(input.via(one))
|
||||
case multiple ⇒
|
||||
val results = Vector.fill(multiple.size)(Sink.publisher[U])
|
||||
val results = Vector.fill(multiple.size)(Sink.publisher[ByteString])
|
||||
val mat =
|
||||
FlowGraph { implicit b ⇒
|
||||
import FlowGraphImplicits._
|
||||
|
||||
val broadcast = Broadcast[T]("transformMultipleInputBroadcast")
|
||||
val broadcast = Broadcast[ByteString]("transformMultipleInputBroadcast")
|
||||
input ~> broadcast
|
||||
(multiple, results).zipped.foreach { (trans, sink) ⇒
|
||||
broadcast ~> Flow[T].transform("transformMultipleElement", trans) ~> sink
|
||||
broadcast ~> trans ~> sink
|
||||
}
|
||||
}.run()
|
||||
results.map(s ⇒ Source(mat.get(s)))
|
||||
}
|
||||
|
||||
def mapEntityError(f: Throwable ⇒ Throwable): RequestEntity ⇒ RequestEntity =
|
||||
_.transformDataBytes(() ⇒ mapErrorTransformer(f))
|
||||
_.transformDataBytes(mapErrorTransformer(f))
|
||||
|
||||
/**
|
||||
* Simple blocking Source backed by an InputStream.
|
||||
|
|
@ -186,13 +177,53 @@ private[http] object StreamUtils {
|
|||
else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[T]], ())
|
||||
}
|
||||
}
|
||||
|
||||
def runStrict(sourceData: ByteString, transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] =
|
||||
Try {
|
||||
transformer match {
|
||||
case Pipe(ops) ⇒
|
||||
if (ops.isEmpty)
|
||||
Some(sourceData)
|
||||
else {
|
||||
@tailrec def tryBuild(remaining: List[AstNode], acc: List[PushPullStage[ByteString, ByteString]]): List[PushPullStage[ByteString, ByteString]] =
|
||||
remaining match {
|
||||
case Nil ⇒ acc.reverse
|
||||
case StageFactory(mkStage, _) :: tail ⇒
|
||||
mkStage() match {
|
||||
case d: PushPullStage[ByteString, ByteString] ⇒
|
||||
tryBuild(tail, d :: acc)
|
||||
case _ ⇒ Nil
|
||||
}
|
||||
case _ ⇒ Nil
|
||||
}
|
||||
|
||||
val strictOps = tryBuild(ops, Nil)
|
||||
if (strictOps.isEmpty)
|
||||
None
|
||||
else {
|
||||
val iter: Iterator[ByteString] = new IteratorInterpreter(Iterator.single(sourceData), strictOps).iterator
|
||||
var byteSize = 0L
|
||||
var result = ByteString.empty
|
||||
var i = 0
|
||||
// note that iter.next() will throw exception if the stream fails, caught by the enclosing Try
|
||||
while (iter.hasNext) {
|
||||
i += 1
|
||||
if (i > maxElements)
|
||||
throw new IllegalArgumentException(s"Too many elements produced by byte transformation, $i was greater than max allowed $maxElements elements")
|
||||
val elem = iter.next()
|
||||
byteSize += elem.size
|
||||
if (byteSize > maxByteSize)
|
||||
throw new IllegalArgumentException(s"Too large data result, $byteSize bytes was greater than max allowed $maxByteSize bytes")
|
||||
result ++= elem
|
||||
}
|
||||
Some(result)
|
||||
}
|
||||
}
|
||||
|
||||
case _ ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] class EnhancedTransformer[T, U](val t: Transformer[T, U]) extends AnyVal {
|
||||
def map[V](f: U ⇒ V): Transformer[T, V] = StreamUtils.mapTransformer(t, f)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -6,10 +6,12 @@ package akka.http
|
|||
|
||||
import language.implicitConversions
|
||||
import language.higherKinds
|
||||
import scala.collection.immutable
|
||||
import java.nio.charset.Charset
|
||||
import com.typesafe.config.Config
|
||||
import akka.stream.{ FlowMaterializer, FlattenStrategy, Transformer }
|
||||
import akka.stream.{ FlowMaterializer, FlattenStrategy }
|
||||
import akka.stream.scaladsl.{ Flow, Source }
|
||||
import akka.stream.stage._
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{ Await, Future }
|
||||
import scala.util.{ Failure, Success }
|
||||
|
|
@ -17,7 +19,6 @@ import scala.util.matching.Regex
|
|||
import akka.event.LoggingAdapter
|
||||
import akka.util.ByteString
|
||||
import akka.actor._
|
||||
import scala.collection.immutable
|
||||
|
||||
package object util {
|
||||
private[http] val UTF8 = Charset.forName("UTF8")
|
||||
|
|
@ -41,8 +42,6 @@ package object util {
|
|||
new EnhancedByteStringTraversableOnce(byteStrings)
|
||||
private[http] implicit def enhanceByteStrings(byteStrings: Source[ByteString]): EnhancedByteStringSource =
|
||||
new EnhancedByteStringSource(byteStrings)
|
||||
private[http] implicit def enhanceTransformer[T, U](transformer: Transformer[T, U]): EnhancedTransformer[T, U] =
|
||||
new EnhancedTransformer(transformer)
|
||||
|
||||
private[http] implicit class SourceWithHeadAndTail[T](val underlying: Source[Source[T]]) extends AnyVal {
|
||||
def headAndTail: Source[(T, Source[T])] =
|
||||
|
|
@ -59,14 +58,18 @@ package object util {
|
|||
private[http] implicit class EnhancedSource[T](val underlying: Source[T]) {
|
||||
def printEvent(marker: String): Source[T] =
|
||||
underlying.transform("transform",
|
||||
() ⇒ new Transformer[T, T] {
|
||||
def onNext(element: T) = {
|
||||
() ⇒ new PushStage[T, T] {
|
||||
override def onPush(element: T, ctx: Context[T]): Directive = {
|
||||
println(s"$marker: $element")
|
||||
element :: Nil
|
||||
ctx.push(element)
|
||||
}
|
||||
override def onTermination(e: Option[Throwable]) = {
|
||||
println(s"$marker: Terminated with error $e")
|
||||
Nil
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[T]): TerminationDirective = {
|
||||
println(s"$marker: Failure $cause")
|
||||
super.onUpstreamFailure(cause, ctx)
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = {
|
||||
println(s"$marker: Terminated")
|
||||
super.onUpstreamFinish(ctx)
|
||||
}
|
||||
})
|
||||
|
||||
|
|
@ -90,10 +93,13 @@ package object util {
|
|||
}
|
||||
}
|
||||
|
||||
private[http] def errorLogger(log: LoggingAdapter, msg: String): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
def onNext(element: ByteString) = element :: Nil
|
||||
override def onError(cause: Throwable): Unit = log.error(cause, msg)
|
||||
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 onUpstreamFailure(cause: Throwable, ctx: Context[ByteString]): TerminationDirective = {
|
||||
log.error(cause, msg)
|
||||
super.onUpstreamFailure(cause, ctx)
|
||||
}
|
||||
}
|
||||
|
||||
private[this] val _identityFunc: Any ⇒ Any = x ⇒ x
|
||||
|
|
|
|||
|
|
@ -38,7 +38,7 @@ object TestClient extends App {
|
|||
} yield response.header[headers.Server]
|
||||
|
||||
def sendRequest(request: HttpRequest, connection: Http.OutgoingConnection): Future[HttpResponse] = {
|
||||
Source(List(HttpRequest() -> 'NoContext))
|
||||
Source.singleton(HttpRequest() -> 'NoContext)
|
||||
.to(Sink(connection.requestSubscriber))
|
||||
.run()
|
||||
Source(connection.responsePublisher).map(_._1).runWith(Sink.head)
|
||||
|
|
|
|||
|
|
@ -252,7 +252,9 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
|||
def renderTo(expected: String): Matcher[HttpRequest] =
|
||||
equal(expected.stripMarginWithNewline("\r\n")).matcher[String] compose { request ⇒
|
||||
val renderer = newRenderer
|
||||
val byteStringSource :: Nil = renderer.onNext(RequestRenderingContext(request, serverAddress))
|
||||
val byteStringSource = Await.result(Source.singleton(RequestRenderingContext(request, serverAddress)).
|
||||
transform("renderer", () ⇒ renderer).
|
||||
runWith(Sink.head), 1.second)
|
||||
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String)
|
||||
Await.result(future, 250.millis)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -402,7 +402,9 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
|
|||
def renderTo(expected: String, close: Boolean): Matcher[ResponseRenderingContext] =
|
||||
equal(expected.stripMarginWithNewline("\r\n") -> close).matcher[(String, Boolean)] compose { ctx ⇒
|
||||
val renderer = newRenderer
|
||||
val byteStringSource :: Nil = renderer.onNext(ctx)
|
||||
val byteStringSource = Await.result(Source.singleton(ctx).
|
||||
transform("renderer", () ⇒ renderer).
|
||||
runWith(Sink.head), 1.second)
|
||||
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String)
|
||||
Await.result(future, 250.millis) -> renderer.isComplete
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,8 +16,8 @@ import akka.util.ByteString
|
|||
import akka.actor.ActorSystem
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.Transformer
|
||||
import akka.http.model.HttpEntity._
|
||||
import akka.http.util.StreamUtils
|
||||
|
||||
class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
||||
val tpe: ContentType = ContentTypes.`application/octet-stream`
|
||||
|
|
@ -120,14 +120,8 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
|||
Await.result(transformed.toStrict(250.millis), 250.millis)
|
||||
}
|
||||
|
||||
def duplicateBytesTransformer(): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
def onNext(bs: ByteString): immutable.Seq[ByteString] =
|
||||
Vector(doubleChars(bs))
|
||||
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ByteString] =
|
||||
Vector(trailer)
|
||||
}
|
||||
def duplicateBytesTransformer(): Flow[ByteString, ByteString] =
|
||||
StreamUtils.byteStringTransformer(doubleChars, () ⇒ trailer)
|
||||
|
||||
def trailer: ByteString = ByteString("--dup")
|
||||
def doubleChars(bs: ByteString): ByteString = ByteString(bs.flatMap(b ⇒ Seq(b, b)): _*)
|
||||
|
|
|
|||
|
|
@ -5,22 +5,22 @@
|
|||
package akka.http.coding
|
||||
|
||||
import akka.http.model.{ HttpRequest, HttpResponse, ResponseEntity, RequestEntity }
|
||||
import akka.stream.Transformer
|
||||
import akka.util.ByteString
|
||||
import akka.stream.scaladsl.Flow
|
||||
|
||||
/** An abstraction to transform data bytes of HttpMessages or HttpEntities */
|
||||
sealed trait DataMapper[T] {
|
||||
def transformDataBytes(t: T, transformer: () ⇒ Transformer[ByteString, ByteString]): T
|
||||
def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString]): T
|
||||
}
|
||||
object DataMapper {
|
||||
implicit val mapRequestEntity: DataMapper[RequestEntity] =
|
||||
new DataMapper[RequestEntity] {
|
||||
def transformDataBytes(t: RequestEntity, transformer: () ⇒ Transformer[ByteString, ByteString]): RequestEntity =
|
||||
def transformDataBytes(t: RequestEntity, transformer: Flow[ByteString, ByteString]): RequestEntity =
|
||||
t.transformDataBytes(transformer)
|
||||
}
|
||||
implicit val mapResponseEntity: DataMapper[ResponseEntity] =
|
||||
new DataMapper[ResponseEntity] {
|
||||
def transformDataBytes(t: ResponseEntity, transformer: () ⇒ Transformer[ByteString, ByteString]): ResponseEntity =
|
||||
def transformDataBytes(t: ResponseEntity, transformer: Flow[ByteString, ByteString]): ResponseEntity =
|
||||
t.transformDataBytes(transformer)
|
||||
}
|
||||
|
||||
|
|
@ -29,7 +29,7 @@ object DataMapper {
|
|||
|
||||
def mapMessage[T, E](entityMapper: DataMapper[E])(mapEntity: (T, E ⇒ E) ⇒ T): DataMapper[T] =
|
||||
new DataMapper[T] {
|
||||
def transformDataBytes(t: T, transformer: () ⇒ Transformer[ByteString, ByteString]): T =
|
||||
def transformDataBytes(t: T, transformer: Flow[ByteString, ByteString]): T =
|
||||
mapEntity(t, entityMapper.transformDataBytes(_, transformer))
|
||||
}
|
||||
}
|
||||
|
|
@ -6,9 +6,9 @@ package akka.http.coding
|
|||
|
||||
import akka.http.model._
|
||||
import akka.http.util.StreamUtils
|
||||
import akka.stream.Transformer
|
||||
import akka.util.ByteString
|
||||
import headers.HttpEncoding
|
||||
import akka.stream.scaladsl.Flow
|
||||
|
||||
trait Decoder {
|
||||
def encoding: HttpEncoding
|
||||
|
|
@ -23,7 +23,7 @@ trait Decoder {
|
|||
|
||||
def newDecompressor: Decompressor
|
||||
|
||||
def newDecodeTransfomer(): Transformer[ByteString, ByteString] = {
|
||||
def newDecodeTransfomer(): Flow[ByteString, ByteString] = {
|
||||
val decompressor = newDecompressor
|
||||
|
||||
def decodeChunk(bytes: ByteString): ByteString = decompressor.decompress(bytes)
|
||||
|
|
|
|||
|
|
@ -7,9 +7,9 @@ package akka.http.coding
|
|||
import java.io.ByteArrayOutputStream
|
||||
import akka.http.model._
|
||||
import akka.http.util.StreamUtils
|
||||
import akka.stream.Transformer
|
||||
import akka.util.ByteString
|
||||
import headers._
|
||||
import akka.stream.scaladsl.Flow
|
||||
|
||||
trait Encoder {
|
||||
def encoding: HttpEncoding
|
||||
|
|
@ -26,7 +26,7 @@ trait Encoder {
|
|||
|
||||
def newCompressor: Compressor
|
||||
|
||||
def newEncodeTransformer(): Transformer[ByteString, ByteString] = {
|
||||
def newEncodeTransformer(): Flow[ByteString, ByteString] = {
|
||||
val compressor = newCompressor
|
||||
|
||||
def encodeChunk(bytes: ByteString): ByteString = compressor.compressAndFlush(bytes)
|
||||
|
|
|
|||
|
|
@ -128,6 +128,7 @@ class GzipDecompressor extends DeflateDecompressor {
|
|||
}
|
||||
|
||||
private def fail(msg: String) = throw new ZipException(msg)
|
||||
|
||||
}
|
||||
|
||||
/** INTERNAL API */
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ trait RangeDirectives {
|
|||
|
||||
class IndexRange(val start: Long, val end: Long) {
|
||||
def length = end - start
|
||||
def apply(entity: UniversalEntity): UniversalEntity = entity.transformDataBytes(length, () ⇒ StreamUtils.sliceBytesTransformer(start, length))
|
||||
def apply(entity: UniversalEntity): UniversalEntity = entity.transformDataBytes(length, StreamUtils.sliceBytesTransformer(start, length))
|
||||
def distance(other: IndexRange) = mergedEnd(other) - mergedStart(other) - (length + other.length)
|
||||
def mergeWith(other: IndexRange) = new IndexRange(mergedStart(other), mergedEnd(other))
|
||||
def contentRange(entityLength: Long) = ContentRange(start, end - 1, entityLength)
|
||||
|
|
@ -73,7 +73,7 @@ trait RangeDirectives {
|
|||
// Therefore, ranges need to be sorted to prevent that some selected ranges already start to accumulate data
|
||||
// but cannot be sent out because another range is blocking the queue.
|
||||
val coalescedRanges = coalesceRanges(iRanges).sortBy(_.start)
|
||||
val bodyPartTransformers = coalescedRanges.map(ir ⇒ () ⇒ StreamUtils.sliceBytesTransformer(ir.start, ir.length)).toVector
|
||||
val bodyPartTransformers = coalescedRanges.map(ir ⇒ StreamUtils.sliceBytesTransformer(ir.start, ir.length)).toVector
|
||||
val bodyPartByteStreams = StreamUtils.transformMultiple(entity.dataBytes, bodyPartTransformers)
|
||||
val bodyParts = (coalescedRanges, bodyPartByteStreams).zipped.map { (range, bytes) ⇒
|
||||
Multipart.ByteRanges.BodyPart(range.contentRange(length), HttpEntity(entity.contentType, range.length, bytes))
|
||||
|
|
|
|||
|
|
@ -15,6 +15,8 @@ import akka.stream.scaladsl._
|
|||
import MediaRanges._
|
||||
import MediaTypes._
|
||||
import HttpCharsets._
|
||||
import akka.stream.impl.fusing.IteratorInterpreter
|
||||
import akka.util.ByteString
|
||||
|
||||
trait MultipartUnmarshallers {
|
||||
|
||||
|
|
@ -67,15 +69,18 @@ trait MultipartUnmarshallers {
|
|||
entity match {
|
||||
case HttpEntity.Strict(ContentType(mediaType: MultipartMediaType, _), data) ⇒
|
||||
val builder = new VectorBuilder[BPS]()
|
||||
(parser.onNext(data) ++ parser.onTermination(None)) foreach {
|
||||
val iter = new IteratorInterpreter[ByteString, BodyPartParser.Output](
|
||||
Iterator.single(data), List(parser)).iterator
|
||||
// note that iter.next() will throw exception if stream fails
|
||||
iter.foreach {
|
||||
case BodyPartStart(headers, createEntity) ⇒
|
||||
val entity = createEntity(Source.empty()) match {
|
||||
case x: HttpEntity.Strict ⇒ x
|
||||
case x ⇒ throw new IllegalStateException("Unexpected entity type from strict BodyPartParser: " + x.getClass.getName)
|
||||
case x ⇒ throw new IllegalStateException("Unexpected entity type from strict BodyPartParser: " + x)
|
||||
}
|
||||
builder += createStrictBodyPart(entity, headers)
|
||||
case ParseError(errorInfo) ⇒ throw new ParsingException(errorInfo)
|
||||
case x ⇒ throw new IllegalStateException(s"Unexpected BodyPartParser result `x` in strict case")
|
||||
case x ⇒ throw new IllegalStateException(s"Unexpected BodyPartParser result $x in strict case")
|
||||
}
|
||||
createStrict(mediaType, builder.result())
|
||||
case _ ⇒
|
||||
|
|
|
|||
|
|
@ -4,10 +4,10 @@
|
|||
package akka.stream.tck
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
import akka.stream.impl.{ Ast, ActorBasedFlowMaterializer }
|
||||
import akka.stream.{ FlowMaterializer, MaterializerSettings }
|
||||
import org.reactivestreams.{ Publisher, Processor }
|
||||
import akka.stream.impl.fusing.Map
|
||||
|
||||
class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
||||
|
||||
|
|
@ -22,7 +22,7 @@ class FusableProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
|||
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
|
||||
|
||||
val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode(
|
||||
Ast.Fused(List(akka.stream.impl.fusing.Map[Int, Int](identity)), "identity"), flowName, 1)
|
||||
Ast.Fused(List(Map[Int, Int](identity)), "identity"), flowName, 1)
|
||||
|
||||
processor.asInstanceOf[Processor[Int, Int]]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,13 +4,14 @@
|
|||
package akka.stream.tck
|
||||
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.impl.ActorBasedFlowMaterializer
|
||||
import akka.stream.impl.Ast
|
||||
import akka.stream.FlowMaterializer
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.reactivestreams.Processor
|
||||
import org.reactivestreams.Publisher
|
||||
import akka.stream.stage.PushStage
|
||||
import akka.stream.stage.Context
|
||||
|
||||
class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
||||
|
||||
|
|
@ -24,13 +25,13 @@ class TransformProcessorTest extends AkkaIdentityProcessorVerification[Int] {
|
|||
|
||||
val flowName = getClass.getSimpleName + "-" + processorCounter.incrementAndGet()
|
||||
|
||||
val mkTransformer = () ⇒
|
||||
new Transformer[Any, Any] {
|
||||
override def onNext(in: Any) = List(in)
|
||||
val mkStage = () ⇒
|
||||
new PushStage[Any, Any] {
|
||||
override def onPush(in: Any, ctx: Context[Any]) = ctx.push(in)
|
||||
}
|
||||
|
||||
val processor = materializer.asInstanceOf[ActorBasedFlowMaterializer].processorForNode(
|
||||
Ast.Transform("transform", mkTransformer), flowName, 1)
|
||||
Ast.StageFactory(mkStage, "transform"), flowName, 1)
|
||||
|
||||
processor.asInstanceOf[Processor[Int, Int]]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,29 +5,25 @@ import akka.dispatch.Foreach;
|
|||
import akka.dispatch.Futures;
|
||||
import akka.dispatch.OnSuccess;
|
||||
import akka.japi.Pair;
|
||||
import akka.japi.Util;
|
||||
import akka.stream.OverflowStrategy;
|
||||
import akka.stream.StreamTest;
|
||||
import akka.stream.Transformer;
|
||||
import akka.stream.stage.*;
|
||||
import akka.stream.javadsl.japi.*;
|
||||
import akka.stream.testkit.AkkaSpec;
|
||||
import akka.testkit.JavaTestKit;
|
||||
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.reactivestreams.Publisher;
|
||||
import scala.Option;
|
||||
import scala.collection.immutable.Seq;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
import scala.runtime.BoxedUnit;
|
||||
import scala.util.Try;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class FlowTest extends StreamTest {
|
||||
|
|
@ -104,37 +100,37 @@ public class FlowTest extends StreamTest {
|
|||
@Test
|
||||
public void mustBeAbleToUseTransform() {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final JavaTestKit probe2 = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7);
|
||||
// duplicate each element, stop after 4 elements, and emit sum to the end
|
||||
Source.from(input).transform("publish", new Creator<Transformer<Integer, Integer>>() {
|
||||
Source.from(input).transform("publish", new Creator<Stage<Integer, Integer>>() {
|
||||
@Override
|
||||
public Transformer<Integer, Integer> create() throws Exception {
|
||||
return new Transformer<Integer, Integer>() {
|
||||
public PushPullStage<Integer, Integer> create() throws Exception {
|
||||
return new StatefulStage<Integer, Integer>() {
|
||||
int sum = 0;
|
||||
int count = 0;
|
||||
|
||||
@Override
|
||||
public scala.collection.immutable.Seq<Integer> onNext(Integer element) {
|
||||
public StageState<Integer, Integer> initial() {
|
||||
return new StageState<Integer, Integer>() {
|
||||
@Override
|
||||
public Directive onPush(Integer element, Context<Integer> ctx) {
|
||||
sum += element;
|
||||
count += 1;
|
||||
return Util.immutableSeq(new Integer[] { element, element });
|
||||
if (count == 4) {
|
||||
return emitAndFinish(Arrays.asList(element, element, sum).iterator(), ctx);
|
||||
} else {
|
||||
return emit(Arrays.asList(element, element).iterator(), ctx);
|
||||
}
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComplete() {
|
||||
return count == 4;
|
||||
public TerminationDirective onUpstreamFinish(Context<Integer> ctx) {
|
||||
return terminationEmit(Collections.singletonList(sum).iterator(), ctx);
|
||||
}
|
||||
|
||||
@Override
|
||||
public scala.collection.immutable.Seq<Integer> onTermination(Option<Throwable> e) {
|
||||
return Util.immutableSingletonSeq(sum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
probe2.getRef().tell("cleanup", ActorRef.noSender());
|
||||
}
|
||||
};
|
||||
}
|
||||
}).foreach(new Procedure<Integer>() {
|
||||
|
|
@ -152,66 +148,6 @@ public class FlowTest extends StreamTest {
|
|||
probe.expectMsgEquals(3);
|
||||
probe.expectMsgEquals(3);
|
||||
probe.expectMsgEquals(6);
|
||||
probe2.expectMsgEquals("cleanup");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void mustBeAbleToUseTransformRecover() {
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
final Iterable<Integer> input = Arrays.asList(0, 1, 2, 3, 4, 5);
|
||||
Source.from(input).map(new Function<Integer, Integer>() {
|
||||
public Integer apply(Integer elem) {
|
||||
if (elem == 4) {
|
||||
throw new IllegalArgumentException("4 not allowed");
|
||||
} else {
|
||||
return elem + elem;
|
||||
}
|
||||
}
|
||||
}).transform("publish", new Creator<Transformer<Integer, String>>() {
|
||||
@Override
|
||||
public Transformer<Integer, String> create() throws Exception {
|
||||
return new Transformer<Integer, String>() {
|
||||
|
||||
@Override
|
||||
public scala.collection.immutable.Seq<String> onNext(Integer element) {
|
||||
return Util.immutableSingletonSeq(element.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public scala.collection.immutable.Seq<String> onTermination(Option<Throwable> e) {
|
||||
if (e.isEmpty()) {
|
||||
return Util.immutableSeq(new String[0]);
|
||||
} else {
|
||||
return Util.immutableSingletonSeq(e.get().getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onError(Throwable e) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComplete() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}).foreach(new Procedure<String>() {
|
||||
public void apply(String elem) {
|
||||
probe.getRef().tell(elem, ActorRef.noSender());
|
||||
}
|
||||
}, materializer);
|
||||
|
||||
probe.expectMsgEquals("0");
|
||||
probe.expectMsgEquals("2");
|
||||
probe.expectMsgEquals("4");
|
||||
probe.expectMsgEquals("6");
|
||||
probe.expectMsgEquals("4 not allowed");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -291,14 +227,19 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
}
|
||||
|
||||
public <In, Out> Creator<Transformer<In, Out>> op() {
|
||||
return new akka.stream.javadsl.japi.Creator<Transformer<In, Out>>() {
|
||||
public <T> Creator<Stage<T, T>> op() {
|
||||
return new akka.stream.javadsl.japi.Creator<Stage<T, T>>() {
|
||||
@Override
|
||||
public Transformer<In, Out> create() throws Exception {
|
||||
return new Transformer<In, Out>() {
|
||||
public PushPullStage<T, T> create() throws Exception {
|
||||
return new PushPullStage<T, T>() {
|
||||
@Override
|
||||
public Seq<Out> onNext(In element) {
|
||||
return Util.immutableSeq(Collections.singletonList((Out) element)); // TODO needs helpers
|
||||
public Directive onPush(T element, Context<T> ctx) {
|
||||
return ctx.push(element);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Directive onPull(Context<T> ctx) {
|
||||
return ctx.pull();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
@ -307,9 +248,9 @@ public class FlowTest extends StreamTest {
|
|||
|
||||
@Test
|
||||
public void mustBeAbleToUseMerge() throws Exception {
|
||||
final Flow<String, String> f1 = Flow.of(String.class).transform("f1", this.<String, String>op()); // javadsl
|
||||
final Flow<String, String> f2 = Flow.of(String.class).transform("f2", this.<String, String>op()); // javadsl
|
||||
final Flow<String, String> f3 = Flow.of(String.class).transform("f2", this.<String, String>op()); // javadsl
|
||||
final Flow<String, String> f1 = Flow.of(String.class).transform("f1", this.<String>op()); // javadsl
|
||||
final Flow<String, String> f2 = Flow.of(String.class).transform("f2", this.<String>op()); // javadsl
|
||||
final Flow<String, String> f3 = Flow.of(String.class).transform("f2", this.<String>op()); // javadsl
|
||||
|
||||
final Source<String> in1 = Source.from(Arrays.asList("a", "b", "c"));
|
||||
final Source<String> in2 = Source.from(Arrays.asList("d", "e", "f"));
|
||||
|
|
|
|||
|
|
@ -329,7 +329,6 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
|
|||
val timeout = 150.millis
|
||||
val a = system.actorOf(timeoutingProps(testActor, timeout))
|
||||
val pub = ActorPublisher(a)
|
||||
watch(a)
|
||||
|
||||
// don't subscribe for `timeout` millis, so it will shut itself down
|
||||
expectMsg("timed-out")
|
||||
|
|
@ -341,6 +340,7 @@ class ActorPublisherSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
expectMsg("cleaned-up")
|
||||
// termination is tiggered by user code
|
||||
watch(a)
|
||||
expectTerminated(a)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,6 +4,7 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.stage._
|
||||
|
||||
trait InterpreterSpecKit extends AkkaSpec {
|
||||
|
||||
|
|
@ -13,25 +14,25 @@ trait InterpreterSpecKit extends AkkaSpec {
|
|||
case class OnNext(elem: Any)
|
||||
case object RequestOne
|
||||
|
||||
private[akka] case class Doubler[T]() extends DeterministicOp[T, T] {
|
||||
private[akka] case class Doubler[T]() extends PushPullStage[T, T] {
|
||||
var oneMore: Boolean = false
|
||||
var lastElem: T = _
|
||||
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
lastElem = elem
|
||||
oneMore = true
|
||||
ctxt.push(elem)
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[T]): Directive = {
|
||||
override def onPull(ctx: Context[T]): Directive = {
|
||||
if (oneMore) {
|
||||
oneMore = false
|
||||
ctxt.push(lastElem)
|
||||
} else ctxt.pull()
|
||||
ctx.push(lastElem)
|
||||
} else ctx.pull()
|
||||
}
|
||||
}
|
||||
|
||||
abstract class TestSetup(ops: Seq[Op[_, _, _, _, _]], forkLimit: Int = 100, overflowToHeap: Boolean = false) {
|
||||
abstract class TestSetup(ops: Seq[Stage[_, _]], forkLimit: Int = 100, overflowToHeap: Boolean = false) {
|
||||
private var lastEvent: Set[Any] = Set.empty
|
||||
|
||||
val upstream = new UpstreamProbe
|
||||
|
|
@ -45,19 +46,19 @@ trait InterpreterSpecKit extends AkkaSpec {
|
|||
result
|
||||
}
|
||||
|
||||
class UpstreamProbe extends BoundaryOp {
|
||||
class UpstreamProbe extends BoundaryStage {
|
||||
|
||||
override def onDownstreamFinish(ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += Cancel
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: BoundaryContext): Directive = {
|
||||
override def onPull(ctx: BoundaryContext): Directive = {
|
||||
lastEvent += RequestOne
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): Directive =
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot push the boundary")
|
||||
|
||||
def onNext(elem: Any): Unit = enter().push(elem)
|
||||
|
|
@ -66,23 +67,23 @@ trait InterpreterSpecKit extends AkkaSpec {
|
|||
|
||||
}
|
||||
|
||||
class DownstreamProbe extends BoundaryOp {
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): Directive = {
|
||||
class DownstreamProbe extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
lastEvent += OnNext(elem)
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += OnComplete
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onFailure(cause: Throwable, ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
lastEvent += OnError(cause)
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: BoundaryContext): Directive =
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("Cannot pull the boundary")
|
||||
|
||||
def requestOne(): Unit = enter().pull()
|
||||
|
|
|
|||
|
|
@ -1,9 +1,12 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.stream.stage._
|
||||
|
||||
class IteratorInterpreterSpec extends AkkaSpec {
|
||||
|
||||
|
|
@ -45,28 +48,32 @@ class IteratorInterpreterSpec extends AkkaSpec {
|
|||
|
||||
"throw exceptions when chain fails" in {
|
||||
val itr = new IteratorInterpreter[Int, Int](List(1, 2, 3).iterator, Seq(
|
||||
new TransitivePullOp[Int, Int] {
|
||||
override def onPush(elem: Int, ctxt: Context[Int]): Directive = {
|
||||
if (elem == 2) ctxt.fail(new ArithmeticException())
|
||||
else ctxt.push(elem)
|
||||
new PushStage[Int, Int] {
|
||||
override def onPush(elem: Int, ctx: Context[Int]): Directive = {
|
||||
if (elem == 2) ctx.fail(new ArithmeticException())
|
||||
else ctx.push(elem)
|
||||
}
|
||||
})).iterator
|
||||
|
||||
itr.next() should be(1)
|
||||
itr.hasNext should be(true)
|
||||
a[ArithmeticException] should be thrownBy { itr.next() }
|
||||
itr.hasNext should be(false)
|
||||
}
|
||||
|
||||
"throw exceptions when op in chain throws" in {
|
||||
val itr = new IteratorInterpreter[Int, Int](List(1, 2, 3).iterator, Seq(
|
||||
new TransitivePullOp[Int, Int] {
|
||||
override def onPush(elem: Int, ctxt: Context[Int]): Directive = {
|
||||
new PushStage[Int, Int] {
|
||||
override def onPush(elem: Int, ctx: Context[Int]): Directive = {
|
||||
if (elem == 2) throw new ArithmeticException()
|
||||
else ctxt.push(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
})).iterator
|
||||
|
||||
itr.next() should be(1)
|
||||
itr.hasNext should be(true)
|
||||
a[ArithmeticException] should be thrownBy { itr.next() }
|
||||
itr.hasNext should be(false)
|
||||
}
|
||||
|
||||
"work with an empty iterator" in {
|
||||
|
|
@ -108,47 +115,47 @@ class IteratorInterpreterSpec extends AkkaSpec {
|
|||
}
|
||||
|
||||
// This op needs an extra pull round to finish
|
||||
case class NaiveTake[T](count: Int) extends DeterministicOp[T, T] {
|
||||
case class NaiveTake[T](count: Int) extends PushPullStage[T, T] {
|
||||
private var left: Int = count
|
||||
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
left -= 1
|
||||
ctxt.push(elem)
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[T]): Directive = {
|
||||
if (left == 0) ctxt.finish()
|
||||
else ctxt.pull()
|
||||
override def onPull(ctx: Context[T]): Directive = {
|
||||
if (left == 0) ctx.finish()
|
||||
else ctx.pull()
|
||||
}
|
||||
}
|
||||
|
||||
case class ByteStringBatcher(threshold: Int, compact: Boolean = true) extends DeterministicOp[ByteString, ByteString] {
|
||||
case class ByteStringBatcher(threshold: Int, compact: Boolean = true) extends PushPullStage[ByteString, ByteString] {
|
||||
require(threshold > 0, "Threshold must be positive")
|
||||
|
||||
private var buf = ByteString.empty
|
||||
private var passthrough = false
|
||||
|
||||
override def onPush(elem: ByteString, ctxt: Context[ByteString]): Directive = {
|
||||
if (passthrough) ctxt.push(elem)
|
||||
override def onPush(elem: ByteString, ctx: Context[ByteString]): Directive = {
|
||||
if (passthrough) ctx.push(elem)
|
||||
else {
|
||||
buf = buf ++ elem
|
||||
if (buf.size >= threshold) {
|
||||
val batch = if (compact) buf.compact else buf
|
||||
passthrough = true
|
||||
buf = ByteString.empty
|
||||
ctxt.push(batch)
|
||||
} else ctxt.pull()
|
||||
ctx.push(batch)
|
||||
} else ctx.pull()
|
||||
}
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[ByteString]): Directive = {
|
||||
if (isFinishing) ctxt.pushAndFinish(buf)
|
||||
else ctxt.pull()
|
||||
override def onPull(ctx: Context[ByteString]): Directive = {
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(buf)
|
||||
else ctx.pull()
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctxt: Context[ByteString]): TerminationDirective = {
|
||||
if (passthrough || buf.isEmpty) ctxt.finish()
|
||||
else ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: Context[ByteString]): TerminationDirective = {
|
||||
if (passthrough || buf.isEmpty) ctx.finish()
|
||||
else ctx.absorbTermination()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -3,10 +3,11 @@
|
|||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.{ OverflowStrategy, Transformer }
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.StreamTestKit.{ PublisherProbe, SubscriberProbe }
|
||||
import akka.stream.stage._
|
||||
|
||||
object FlowGraphCompileSpec {
|
||||
class Fruit
|
||||
|
|
@ -18,9 +19,10 @@ class FlowGraphCompileSpec extends AkkaSpec {
|
|||
|
||||
implicit val mat = FlowMaterializer()
|
||||
|
||||
def op[In, Out]: () ⇒ Transformer[In, Out] = { () ⇒
|
||||
new Transformer[In, Out] {
|
||||
override def onNext(elem: In) = List(elem.asInstanceOf[Out])
|
||||
def op[In, Out]: () ⇒ PushStage[In, Out] = { () ⇒
|
||||
new PushStage[In, Out] {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive =
|
||||
ctx.push(elem.asInstanceOf[Out])
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -4,13 +4,10 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.stream.impl.fusing.{ Op, ActorInterpreter }
|
||||
|
||||
import akka.stream.stage.Stage
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import akka.actor._
|
||||
import akka.stream.{ TransformerLike, MaterializerSettings }
|
||||
import akka.stream.FlowMaterializer
|
||||
|
|
@ -22,6 +19,8 @@ import akka.testkit._
|
|||
import akka.testkit.TestEvent.{ UnMute, Mute }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.reactivestreams.{ Processor, Subscriber, Publisher }
|
||||
import akka.stream.impl.fusing.ActorInterpreter
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
object FlowSpec {
|
||||
class Fruit
|
||||
|
|
@ -32,25 +31,9 @@ object FlowSpec {
|
|||
|
||||
case class BrokenMessage(msg: String)
|
||||
|
||||
class BrokenTransformProcessorImpl(
|
||||
_settings: MaterializerSettings,
|
||||
transformer: TransformerLike[Any, Any],
|
||||
brokenMessage: Any) extends TransformProcessorImpl(_settings, transformer) {
|
||||
|
||||
import akka.stream.actor.ActorSubscriberMessage._
|
||||
|
||||
override protected[akka] def aroundReceive(receive: Receive, msg: Any) = {
|
||||
msg match {
|
||||
case OnNext(m) if m == brokenMessage ⇒
|
||||
throw new NullPointerException(s"I'm so broken [$m]")
|
||||
case _ ⇒ super.aroundReceive(receive, msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class BrokenActorInterpreter(
|
||||
_settings: MaterializerSettings,
|
||||
_ops: Seq[Op[_, _, _, _, _]],
|
||||
_ops: Seq[Stage[_, _]],
|
||||
brokenMessage: Any)
|
||||
extends ActorInterpreter(_settings, _ops) {
|
||||
|
||||
|
|
@ -76,7 +59,6 @@ object FlowSpec {
|
|||
|
||||
override def processorForNode[In, Out](op: AstNode, flowName: String, n: Int): Processor[In, Out] = {
|
||||
val props = op match {
|
||||
case t: Transform ⇒ Props(new BrokenTransformProcessorImpl(settings, t.mkTransformer(), brokenMessage))
|
||||
case f: Fused ⇒ Props(new BrokenActorInterpreter(settings, f.ops, brokenMessage)).withDispatcher(settings.dispatcher)
|
||||
case Map(f) ⇒ Props(new BrokenActorInterpreter(settings, List(fusing.Map(f)), brokenMessage))
|
||||
case Filter(p) ⇒ Props(new BrokenActorInterpreter(settings, List(fusing.Filter(p)), brokenMessage))
|
||||
|
|
@ -639,6 +621,6 @@ class FlowSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.rece
|
|||
}
|
||||
}
|
||||
|
||||
object TestException extends RuntimeException
|
||||
object TestException extends RuntimeException with NoStackTrace
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,15 +6,14 @@ package akka.stream.scaladsl
|
|||
import scala.collection.immutable.Seq
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.testkit.{ AkkaSpec, StreamTestKit }
|
||||
import akka.testkit.{ EventFilter, TestProbe }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.stream.stage._
|
||||
|
||||
class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||
class FlowStageSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.debug.receive=off\nakka.loglevel=INFO")) {
|
||||
|
||||
val settings = MaterializerSettings(system)
|
||||
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||
|
|
@ -26,11 +25,11 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
"produce one-to-one transformation as expected" in {
|
||||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
transform("transform", () ⇒ new PushStage[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = {
|
||||
tot += elem
|
||||
List(tot)
|
||||
ctx.push(tot)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
|
|
@ -49,12 +48,23 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
"produce one-to-several transformation as expected" in {
|
||||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
transform("transform", () ⇒ new StatefulStage[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
|
||||
lazy val waitForNext = new State {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = {
|
||||
tot += elem
|
||||
Vector.fill(elem)(tot)
|
||||
emit(Iterator.fill(elem)(tot), ctx)
|
||||
}
|
||||
}
|
||||
|
||||
override def initial = waitForNext
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Int]): TerminationDirective = {
|
||||
if (current eq waitForNext) ctx.finish()
|
||||
else ctx.absorbTermination()
|
||||
}
|
||||
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
|
|
@ -75,15 +85,14 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
"produce dropping transformation as expected" in {
|
||||
val p = Source(List(1, 2, 3, 4)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
transform("transform", () ⇒ new PushStage[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = {
|
||||
tot += elem
|
||||
if (elem % 2 == 0) {
|
||||
Nil
|
||||
} else {
|
||||
List(tot)
|
||||
}
|
||||
if (elem % 2 == 0)
|
||||
ctx.pull()
|
||||
else
|
||||
ctx.push(tot)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
|
|
@ -102,18 +111,18 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
"produce multi-step transformation as expected" in {
|
||||
val p = Source(List("a", "bc", "def")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[String, Int] {
|
||||
transform("transform", () ⇒ new PushStage[String, Int] {
|
||||
var concat = ""
|
||||
override def onNext(elem: String) = {
|
||||
override def onPush(elem: String, ctx: Context[Int]) = {
|
||||
concat += elem
|
||||
List(concat.length)
|
||||
ctx.push(concat.length)
|
||||
}
|
||||
}).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
transform("transform", () ⇒ new PushStage[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(length: Int) = {
|
||||
override def onPush(length: Int, ctx: Context[Int]) = {
|
||||
tot += length
|
||||
List(tot)
|
||||
ctx.push(tot)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.fanoutPublisher(2, 2))
|
||||
|
|
@ -138,16 +147,19 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
c2.expectComplete()
|
||||
}
|
||||
|
||||
"invoke onComplete when done" in {
|
||||
"support emit onUpstreamFinish" in {
|
||||
val p = Source(List("a")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[String, String] {
|
||||
transform("transform", () ⇒ new StatefulStage[String, String] {
|
||||
var s = ""
|
||||
override def onNext(element: String) = {
|
||||
override def initial = new State {
|
||||
override def onPush(element: String, ctx: Context[String]) = {
|
||||
s += element
|
||||
Nil
|
||||
ctx.pull()
|
||||
}
|
||||
override def onTermination(e: Option[Throwable]) = List(s + "B")
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[String]) =
|
||||
terminationEmit(Iterator.single(s + "B"), ctx)
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val c = StreamTestKit.SubscriberProbe[String]()
|
||||
|
|
@ -158,84 +170,18 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
c.expectComplete()
|
||||
}
|
||||
|
||||
"invoke cleanup when done" in {
|
||||
val cleanupProbe = TestProbe()
|
||||
val p = Source(List("a")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[String, String] {
|
||||
var s = ""
|
||||
override def onNext(element: String) = {
|
||||
s += element
|
||||
Nil
|
||||
}
|
||||
override def onTermination(e: Option[Throwable]) = List(s + "B")
|
||||
override def cleanup() = cleanupProbe.ref ! s
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val c = StreamTestKit.SubscriberProbe[String]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(1)
|
||||
c.expectNext("aB")
|
||||
c.expectComplete()
|
||||
cleanupProbe.expectMsg("a")
|
||||
}
|
||||
|
||||
"invoke cleanup when done consume" in {
|
||||
val cleanupProbe = TestProbe()
|
||||
val p = Source(List("a")).runWith(Sink.publisher)
|
||||
Source(p).
|
||||
transform("transform", () ⇒ new Transformer[String, String] {
|
||||
var s = "x"
|
||||
override def onNext(element: String) = {
|
||||
s = element
|
||||
List(element)
|
||||
}
|
||||
override def cleanup() = cleanupProbe.ref ! s
|
||||
}).
|
||||
to(Sink.ignore).run()
|
||||
cleanupProbe.expectMsg("a")
|
||||
}
|
||||
|
||||
"invoke cleanup when done after error" in {
|
||||
val cleanupProbe = TestProbe()
|
||||
val p = Source(List("a", "b", "c")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[String, String] {
|
||||
var s = ""
|
||||
override def onNext(in: String) = {
|
||||
if (in == "b") {
|
||||
throw new IllegalArgumentException("Not b") with NoStackTrace
|
||||
} else {
|
||||
val out = s + in
|
||||
s += in.toUpperCase
|
||||
List(out)
|
||||
}
|
||||
}
|
||||
override def onTermination(e: Option[Throwable]) = List(s + "B")
|
||||
override def cleanup() = cleanupProbe.ref ! s
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val c = StreamTestKit.SubscriberProbe[String]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(1)
|
||||
c.expectNext("a")
|
||||
s.request(1)
|
||||
c.expectError()
|
||||
cleanupProbe.expectMsg("A")
|
||||
}
|
||||
|
||||
"allow cancellation using isComplete" in {
|
||||
"allow early finish" in {
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
transform("transform", () ⇒ new PushStage[Int, Int] {
|
||||
var s = ""
|
||||
override def onNext(element: Int) = {
|
||||
override def onPush(element: Int, ctx: Context[Int]) = {
|
||||
s += element
|
||||
List(element)
|
||||
if (s == "1")
|
||||
ctx.pushAndFinish(element)
|
||||
else
|
||||
ctx.push(element)
|
||||
}
|
||||
override def isComplete = s == "1"
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription
|
||||
|
|
@ -250,44 +196,17 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
proc.expectCancellation()
|
||||
}
|
||||
|
||||
"call onComplete after isComplete signaled completion" in {
|
||||
val cleanupProbe = TestProbe()
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
var s = ""
|
||||
override def onNext(element: Int) = {
|
||||
s += element
|
||||
List(element)
|
||||
}
|
||||
override def isComplete = s == "1"
|
||||
override def onTermination(e: Option[Throwable]) = List(s.length + 10)
|
||||
override def cleanup() = cleanupProbe.ref ! s
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription
|
||||
val c = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(10)
|
||||
proc.sendNext(1)
|
||||
proc.sendNext(2)
|
||||
c.expectNext(1)
|
||||
c.expectNext(11)
|
||||
c.expectComplete()
|
||||
proc.expectCancellation()
|
||||
cleanupProbe.expectMsg("1")
|
||||
}
|
||||
|
||||
"report error when exception is thrown" in {
|
||||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = {
|
||||
transform("transform", () ⇒ new StatefulStage[Int, Int] {
|
||||
override def initial = new State {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = {
|
||||
if (elem == 2) {
|
||||
throw new IllegalArgumentException("two not allowed")
|
||||
} else {
|
||||
List(elem, elem)
|
||||
emit(Iterator(elem, elem), ctx)
|
||||
}
|
||||
}
|
||||
}
|
||||
}).
|
||||
|
|
@ -304,11 +223,41 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
}
|
||||
}
|
||||
|
||||
"support emit of final elements when onUpstreamFailure" in {
|
||||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
map(elem ⇒ if (elem == 2) throw new IllegalArgumentException("two not allowed") else elem).
|
||||
transform("transform", () ⇒ new StatefulStage[Int, Int] {
|
||||
override def initial = new State {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ctx.push(elem)
|
||||
}
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[Int]) = {
|
||||
terminationEmit(Iterator(100, 101), ctx)
|
||||
}
|
||||
}).
|
||||
filter(elem ⇒ elem != 1). // it's undefined if element 1 got through before the error or not
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
EventFilter[IllegalArgumentException]("two not allowed") intercept {
|
||||
subscription.request(100)
|
||||
subscriber.expectNext(100)
|
||||
subscriber.expectNext(101)
|
||||
subscriber.expectComplete()
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
}
|
||||
}
|
||||
|
||||
"support cancel as expected" in {
|
||||
val p = Source(List(1, 2, 3)).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = List(elem, elem)
|
||||
transform("transform", () ⇒ new StatefulStage[Int, Int] {
|
||||
override def initial = new State {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) =
|
||||
emit(Iterator(elem, elem), ctx)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
|
|
@ -326,9 +275,12 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
"support producing elements from empty inputs" in {
|
||||
val p = Source(List.empty[Int]).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = Nil
|
||||
override def onTermination(e: Option[Throwable]) = List(1, 2, 3)
|
||||
transform("transform", () ⇒ new StatefulStage[Int, Int] {
|
||||
override def initial = new State {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = ctx.pull()
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[Int]) =
|
||||
terminationEmit(Iterator(1, 2, 3), ctx)
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
|
|
@ -344,25 +296,23 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
|
||||
"support converting onComplete into onError" in {
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
Source(List(5, 1, 2, 3)).transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
Source(List(5, 1, 2, 3)).transform("transform", () ⇒ new PushStage[Int, Int] {
|
||||
var expectedNumberOfElements: Option[Int] = None
|
||||
var count = 0
|
||||
override def onNext(elem: Int) =
|
||||
override def onPush(elem: Int, ctx: Context[Int]) =
|
||||
if (expectedNumberOfElements.isEmpty) {
|
||||
expectedNumberOfElements = Some(elem)
|
||||
Nil
|
||||
ctx.pull()
|
||||
} else {
|
||||
count += 1
|
||||
List(elem)
|
||||
ctx.push(elem)
|
||||
}
|
||||
override def onTermination(err: Option[Throwable]) = err match {
|
||||
case Some(e) ⇒ Nil
|
||||
case None ⇒
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Int]) =
|
||||
expectedNumberOfElements match {
|
||||
case Some(expected) if count != expected ⇒
|
||||
throw new RuntimeException(s"Expected $expected, got $count") with NoStackTrace
|
||||
case _ ⇒ Nil
|
||||
}
|
||||
case _ ⇒ ctx.finish()
|
||||
}
|
||||
}).to(Sink(subscriber)).run()
|
||||
|
||||
|
|
@ -377,12 +327,12 @@ class FlowTransformSpec extends AkkaSpec(ConfigFactory.parseString("akka.actor.d
|
|||
|
||||
"be safe to reuse" in {
|
||||
val flow = Source(1 to 3).transform("transform", () ⇒
|
||||
new Transformer[Int, Int] {
|
||||
new PushStage[Int, Int] {
|
||||
var count = 0
|
||||
|
||||
override def onNext(elem: Int): Seq[Int] = {
|
||||
override def onPush(elem: Int, ctx: Context[Int]) = {
|
||||
count += 1
|
||||
List(count)
|
||||
ctx.push(count)
|
||||
}
|
||||
})
|
||||
|
||||
|
|
@ -1,390 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.scaladsl
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.Failure
|
||||
import scala.util.Success
|
||||
import scala.util.Try
|
||||
import scala.util.control.NoStackTrace
|
||||
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
import akka.stream.testkit.StreamTestKit
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
object FlowTransformRecoverSpec {
|
||||
abstract class TryRecoveryTransformer[T, U] extends Transformer[T, U] {
|
||||
def onNext(element: Try[T]): immutable.Seq[U]
|
||||
|
||||
override def onNext(element: T): immutable.Seq[U] = onNext(Success(element))
|
||||
override def onError(cause: Throwable) = ()
|
||||
override def onTermination(cause: Option[Throwable]): immutable.Seq[U] = cause match {
|
||||
case None ⇒ Nil
|
||||
case Some(e) ⇒ onNext(Failure(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class FlowTransformRecoverSpec extends AkkaSpec {
|
||||
import FlowTransformRecoverSpec._
|
||||
|
||||
val settings = MaterializerSettings(system)
|
||||
.withInputBuffer(initialSize = 2, maxSize = 2)
|
||||
.withFanOutBuffer(initialSize = 2, maxSize = 2)
|
||||
|
||||
implicit val materializer = FlowMaterializer(settings)
|
||||
|
||||
"A Flow with transformRecover operations" must {
|
||||
"produce one-to-one transformation as expected" in {
|
||||
val p = Source(1 to 3).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
tot += elem
|
||||
List(tot)
|
||||
}
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = e match {
|
||||
case None ⇒ Nil
|
||||
case Some(_) ⇒ List(-1)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
subscription.request(2)
|
||||
subscriber.expectNext(3)
|
||||
subscriber.expectNext(6)
|
||||
subscriber.expectComplete()
|
||||
}
|
||||
|
||||
"produce one-to-several transformation as expected" in {
|
||||
val p = Source(1 to 3).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
tot += elem
|
||||
Vector.fill(elem)(tot)
|
||||
}
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = e match {
|
||||
case None ⇒ Nil
|
||||
case Some(_) ⇒ List(-1)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(4)
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectNext(3)
|
||||
subscriber.expectNext(3)
|
||||
subscriber.expectNext(6)
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
subscription.request(100)
|
||||
subscriber.expectNext(6)
|
||||
subscriber.expectNext(6)
|
||||
subscriber.expectComplete()
|
||||
}
|
||||
|
||||
"produce dropping transformation as expected" in {
|
||||
val p = Source(1 to 4).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(elem: Int) = {
|
||||
tot += elem
|
||||
if (elem % 2 == 0) Nil else List(tot)
|
||||
}
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = e match {
|
||||
case None ⇒ Nil
|
||||
case Some(_) ⇒ List(-1)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(6)
|
||||
subscription.request(1)
|
||||
subscriber.expectComplete()
|
||||
}
|
||||
|
||||
"produce multi-step transformation as expected" in {
|
||||
val p = Source(List("a", "bc", "def")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new TryRecoveryTransformer[String, Int] {
|
||||
var concat = ""
|
||||
override def onNext(element: Try[String]) = {
|
||||
concat += element
|
||||
List(concat.length)
|
||||
}
|
||||
}).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
var tot = 0
|
||||
override def onNext(length: Int) = {
|
||||
tot += length
|
||||
List(tot)
|
||||
}
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = e match {
|
||||
case None ⇒ Nil
|
||||
case Some(_) ⇒ List(-1)
|
||||
}
|
||||
}).runWith(Sink.fanoutPublisher(1, 1))
|
||||
val c1 = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c1)
|
||||
val sub1 = c1.expectSubscription()
|
||||
val c2 = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c2)
|
||||
val sub2 = c2.expectSubscription()
|
||||
sub1.request(1)
|
||||
sub2.request(2)
|
||||
c1.expectNext(10)
|
||||
c2.expectNext(10)
|
||||
c2.expectNext(31)
|
||||
c1.expectNoMsg(200.millis)
|
||||
sub1.request(2)
|
||||
sub2.request(2)
|
||||
c1.expectNext(31)
|
||||
c1.expectNext(64)
|
||||
c2.expectNext(64)
|
||||
c1.expectComplete()
|
||||
c2.expectComplete()
|
||||
}
|
||||
|
||||
"invoke onComplete when done" in {
|
||||
val p = Source(List("a")).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new TryRecoveryTransformer[String, String] {
|
||||
var s = ""
|
||||
override def onNext(element: Try[String]) = {
|
||||
s += element
|
||||
Nil
|
||||
}
|
||||
override def onTermination(e: Option[Throwable]) = List(s + "B")
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val c = StreamTestKit.SubscriberProbe[String]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(1)
|
||||
c.expectNext("Success(a)B")
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"allow cancellation using isComplete" in {
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new TryRecoveryTransformer[Int, Int] {
|
||||
var s = ""
|
||||
override def onNext(element: Try[Int]) = {
|
||||
s += element
|
||||
List(element.get)
|
||||
}
|
||||
override def isComplete = s == "Success(1)"
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription
|
||||
val c = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(10)
|
||||
proc.sendNext(1)
|
||||
proc.sendNext(2)
|
||||
c.expectNext(1)
|
||||
c.expectComplete()
|
||||
proc.expectCancellation()
|
||||
}
|
||||
|
||||
"call onComplete after isComplete signaled completion" in {
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new TryRecoveryTransformer[Int, Int] {
|
||||
var s = ""
|
||||
override def onNext(element: Try[Int]) = {
|
||||
s += element
|
||||
List(element.get)
|
||||
}
|
||||
override def isComplete = s == "Success(1)"
|
||||
override def onTermination(e: Option[Throwable]) = List(s.length + 10)
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription
|
||||
val c = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(10)
|
||||
proc.sendNext(1)
|
||||
proc.sendNext(2)
|
||||
c.expectNext(1)
|
||||
c.expectNext(20)
|
||||
c.expectComplete()
|
||||
proc.expectCancellation()
|
||||
}
|
||||
|
||||
"report error when exception is thrown" in {
|
||||
val p = Source(1 to 3).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = {
|
||||
if (elem == 2) throw new IllegalArgumentException("two not allowed")
|
||||
else List(elem, elem)
|
||||
}
|
||||
override def onError(e: Throwable) = List(-1)
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
EventFilter[IllegalArgumentException]("two not allowed") intercept {
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
subscription.request(100)
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectError().getMessage should be("two not allowed")
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
}
|
||||
}
|
||||
|
||||
"report error after emitted elements" in {
|
||||
EventFilter[IllegalArgumentException]("two not allowed") intercept {
|
||||
val p2 = Source(1 to 3).
|
||||
mapConcat { elem ⇒
|
||||
if (elem == 2) throw new IllegalArgumentException("two not allowed")
|
||||
else (1 to 5).map(elem * 100 + _)
|
||||
}.
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = List(elem)
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = e match {
|
||||
case None ⇒ Nil
|
||||
case Some(_) ⇒ List(-1, -2, -3)
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(101)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(102)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(103)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(104)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(105)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
|
||||
subscription.request(1)
|
||||
subscriber.expectNext(-1)
|
||||
subscriber.expectNoMsg(100.millis)
|
||||
subscription.request(10)
|
||||
subscriber.expectNext(-2)
|
||||
subscriber.expectNext(-3)
|
||||
subscriber.expectComplete()
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
}
|
||||
}
|
||||
|
||||
case class TE(message: String) extends RuntimeException(message) with NoStackTrace
|
||||
|
||||
"transform errors in sequence with normal messages" in {
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, String] {
|
||||
var s = ""
|
||||
override def onNext(element: Int) = {
|
||||
s += element.toString
|
||||
List(s)
|
||||
}
|
||||
override def onError(ex: Throwable) = ()
|
||||
override def onTermination(ex: Option[Throwable]) = {
|
||||
ex match {
|
||||
case None ⇒ Nil
|
||||
case Some(e) ⇒
|
||||
s += e.getMessage
|
||||
List(s)
|
||||
}
|
||||
}
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription()
|
||||
val c = StreamTestKit.SubscriberProbe[String]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
proc.sendNext(0)
|
||||
proc.sendError(TE("1"))
|
||||
// Request late to prove the in-sequence nature
|
||||
s.request(10)
|
||||
c.expectNext("0")
|
||||
c.expectNext("01")
|
||||
c.expectComplete()
|
||||
}
|
||||
|
||||
"forward errors when received and thrown" in {
|
||||
val p = StreamTestKit.PublisherProbe[Int]()
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(in: Int) = List(in)
|
||||
override def onError(e: Throwable) = throw e
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val proc = p.expectSubscription()
|
||||
val c = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(c)
|
||||
val s = c.expectSubscription()
|
||||
s.request(10)
|
||||
EventFilter[TE](occurrences = 1) intercept {
|
||||
proc.sendError(TE("1"))
|
||||
c.expectError(TE("1"))
|
||||
}
|
||||
}
|
||||
|
||||
"support cancel as expected" in {
|
||||
val p = Source(1 to 3).runWith(Sink.publisher)
|
||||
val p2 = Source(p).
|
||||
transform("transform", () ⇒ new Transformer[Int, Int] {
|
||||
override def onNext(elem: Int) = List(elem, elem)
|
||||
override def onError(e: Throwable) = List(-1)
|
||||
}).
|
||||
runWith(Sink.publisher)
|
||||
val subscriber = StreamTestKit.SubscriberProbe[Int]()
|
||||
p2.subscribe(subscriber)
|
||||
val subscription = subscriber.expectSubscription()
|
||||
subscription.request(2)
|
||||
subscriber.expectNext(1)
|
||||
subscription.cancel()
|
||||
subscriber.expectNext(1)
|
||||
subscriber.expectNoMsg(500.millis)
|
||||
subscription.request(2)
|
||||
subscriber.expectNoMsg(200.millis)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -20,7 +20,7 @@ class OptimizingActorBasedFlowMaterializerSpec extends AkkaSpec with ImplicitSen
|
|||
val f = Source(1 to 100).
|
||||
drop(4).
|
||||
drop(5).
|
||||
transform("identity", () ⇒ FlowOps.identityTransformer).
|
||||
transform("identity", () ⇒ FlowOps.identityStage).
|
||||
filter(_ % 2 == 0).
|
||||
map(_ * 2).
|
||||
map(identity).
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import scala.collection.{ immutable, mutable }
|
|||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
/**
|
||||
* [[Transformer]] with support for scheduling keyed (named) timer events.
|
||||
* Transformer with support for scheduling keyed (named) timer events.
|
||||
*/
|
||||
abstract class TimerTransformer[-T, +U] extends TransformerLike[T, U] {
|
||||
import TimerTransformer._
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ abstract class TransformerLike[-T, +U] {
|
|||
* to produce a (possibly empty) sequence of elements in response to the
|
||||
* end-of-stream event.
|
||||
*
|
||||
* This method is only called if [[Transformer#onError]] does not throw an exception. The default implementation
|
||||
* of [[Transformer#onError]] throws the received cause forcing the error to propagate downstream immediately.
|
||||
* This method is only called if [[#onError]] does not throw an exception. The default implementation
|
||||
* of [[#onError]] throws the received cause forcing the error to propagate downstream immediately.
|
||||
*
|
||||
* @param e Contains a non-empty option with the error causing the termination or an empty option
|
||||
* if the Transformer was completed normally
|
||||
|
|
@ -34,7 +34,7 @@ abstract class TransformerLike[-T, +U] {
|
|||
|
||||
/**
|
||||
* Invoked when failure is signaled from upstream. If this method throws an exception, then onError is immediately
|
||||
* propagated downstream. If this method completes normally then [[Transformer#onTermination]] is invoked as a final
|
||||
* propagated downstream. If this method completes normally then [[#onTermination]] is invoked as a final
|
||||
* step, passing the original cause.
|
||||
*/
|
||||
def onError(cause: Throwable): Unit = throw cause
|
||||
|
|
@ -46,15 +46,3 @@ abstract class TransformerLike[-T, +U] {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* General interface for stream transformation.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[Transformer]] instance with
|
||||
* ordinary instance variables. The [[Transformer]] is executed by an actor and
|
||||
* therefore you don not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
*
|
||||
* @see [[akka.stream.scaladsl.Flow#transform]]
|
||||
* @see [[akka.stream.javadsl.Flow#transform]]
|
||||
*/
|
||||
abstract class Transformer[-T, +U] extends TransformerLike[T, U]
|
||||
|
|
|
|||
|
|
@ -8,9 +8,9 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
import scala.concurrent.duration._
|
||||
import scala.language.implicitConversions
|
||||
import scala.language.existentials
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.stage._
|
||||
|
||||
/**
|
||||
* Provides operations needed to implement the `timed` DSL
|
||||
|
|
@ -99,41 +99,49 @@ object Timed extends TimedOps with TimedIntervalBetweenOps {
|
|||
}
|
||||
}
|
||||
|
||||
final class StartTimedFlow[T](ctx: TimedFlowContext) extends Transformer[T, T] {
|
||||
final class StartTimedFlow[T](timedContext: TimedFlowContext) extends PushStage[T, T] {
|
||||
private var started = false
|
||||
|
||||
override def onNext(element: T) = {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
if (!started) {
|
||||
ctx.start()
|
||||
timedContext.start()
|
||||
started = true
|
||||
}
|
||||
|
||||
immutable.Seq(element)
|
||||
ctx.push(elem)
|
||||
}
|
||||
}
|
||||
|
||||
final class StopTimed[T](ctx: TimedFlowContext, _onComplete: FiniteDuration ⇒ Unit) extends Transformer[T, T] {
|
||||
final class StopTimed[T](timedContext: TimedFlowContext, _onComplete: FiniteDuration ⇒ Unit) extends PushStage[T, T] {
|
||||
|
||||
override def cleanup() {
|
||||
val d = ctx.stop()
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = ctx.push(elem)
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[T]): TerminationDirective = {
|
||||
stopTime()
|
||||
ctx.fail(cause)
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[T]): TerminationDirective = {
|
||||
stopTime()
|
||||
ctx.finish()
|
||||
}
|
||||
private def stopTime() {
|
||||
val d = timedContext.stop()
|
||||
_onComplete(d)
|
||||
}
|
||||
|
||||
override def onNext(element: T) = immutable.Seq(element)
|
||||
}
|
||||
|
||||
final class TimedIntervalTransformer[T](matching: T ⇒ Boolean, onInterval: FiniteDuration ⇒ Unit) extends Transformer[T, T] {
|
||||
final class TimedIntervalTransformer[T](matching: T ⇒ Boolean, onInterval: FiniteDuration ⇒ Unit) extends PushStage[T, T] {
|
||||
private var prevNanos = 0L
|
||||
private var matched = 0L
|
||||
|
||||
override def onNext(in: T): immutable.Seq[T] = {
|
||||
if (matching(in)) {
|
||||
val d = updateInterval(in)
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
if (matching(elem)) {
|
||||
val d = updateInterval(elem)
|
||||
|
||||
if (matched > 1)
|
||||
onInterval(d)
|
||||
}
|
||||
immutable.Seq(in)
|
||||
ctx.push(elem)
|
||||
}
|
||||
|
||||
private def updateInterval(in: T): FiniteDuration = {
|
||||
|
|
|
|||
|
|
@ -7,18 +7,17 @@ import java.util.concurrent.atomic.AtomicLong
|
|||
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.Logging
|
||||
import akka.stream.impl.fusing.{ ActorInterpreter, Op }
|
||||
|
||||
import akka.stream.impl.fusing.ActorInterpreter
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ ExecutionContext, Await, Future }
|
||||
|
||||
import akka.actor._
|
||||
import akka.stream.{ FlowMaterializer, MaterializerSettings, OverflowStrategy, TimerTransformer, Transformer }
|
||||
import akka.stream.{ FlowMaterializer, MaterializerSettings, OverflowStrategy, TimerTransformer }
|
||||
import akka.stream.MaterializationException
|
||||
import akka.stream.actor.ActorSubscriber
|
||||
import akka.stream.impl.Zip.ZipAs
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.stage._
|
||||
import akka.pattern.ask
|
||||
import org.reactivestreams.{ Processor, Publisher, Subscriber }
|
||||
|
||||
|
|
@ -29,20 +28,16 @@ private[akka] object Ast {
|
|||
sealed abstract class AstNode {
|
||||
def name: String
|
||||
}
|
||||
// FIXME Replace with Operate
|
||||
final case class Transform(name: String, mkTransformer: () ⇒ Transformer[Any, Any]) extends AstNode
|
||||
// FIXME Replace with Operate
|
||||
final case class TimerTransform(name: String, mkTransformer: () ⇒ TimerTransformer[Any, Any]) extends AstNode
|
||||
|
||||
final case class Operate(mkOp: () ⇒ fusing.Op[_, _, _, _, _]) extends AstNode {
|
||||
override def name = "operate"
|
||||
}
|
||||
final case class TimerTransform(mkStage: () ⇒ TimerTransformer[Any, Any], override val name: String) extends AstNode
|
||||
|
||||
final case class StageFactory(mkStage: () ⇒ Stage[_, _], override val name: String) extends AstNode
|
||||
|
||||
object Fused {
|
||||
def apply(ops: immutable.Seq[Op[_, _, _, _, _]]): Fused =
|
||||
def apply(ops: immutable.Seq[Stage[_, _]]): Fused =
|
||||
Fused(ops, ops.map(x ⇒ Logging.simpleName(x).toLowerCase).mkString("+")) //FIXME change to something more performant for name
|
||||
}
|
||||
final case class Fused(ops: immutable.Seq[Op[_, _, _, _, _]], override val name: String) extends AstNode
|
||||
final case class Fused(ops: immutable.Seq[Stage[_, _]], override val name: String) extends AstNode
|
||||
|
||||
final case class Map(f: Any ⇒ Any) extends AstNode { override def name = "map" }
|
||||
|
||||
|
|
@ -197,7 +192,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
|
|||
//FIXME Optimize the implementation of the optimizer (no joke)
|
||||
// AstNodes are in reverse order, Fusable Ops are in order
|
||||
private[this] final def optimize(ops: List[Ast.AstNode]): (List[Ast.AstNode], Int) = {
|
||||
@tailrec def analyze(rest: List[Ast.AstNode], optimized: List[Ast.AstNode], fuseCandidates: List[fusing.Op[_, _, _, _, _]]): (List[Ast.AstNode], Int) = {
|
||||
@tailrec def analyze(rest: List[Ast.AstNode], optimized: List[Ast.AstNode], fuseCandidates: List[Stage[_, _]]): (List[Ast.AstNode], Int) = {
|
||||
|
||||
//The `verify` phase
|
||||
def verify(rest: List[Ast.AstNode], orig: List[Ast.AstNode]): List[Ast.AstNode] =
|
||||
|
|
@ -245,10 +240,10 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
|
|||
}
|
||||
|
||||
// Tries to squeeze AstNode into a single fused pipeline
|
||||
def ast2op(head: Ast.AstNode, prev: List[fusing.Op[_, _, _, _, _]]): List[fusing.Op[_, _, _, _, _]] =
|
||||
def ast2op(head: Ast.AstNode, prev: List[Stage[_, _]]): List[Stage[_, _]] =
|
||||
head match {
|
||||
// Always-on below
|
||||
case Ast.Operate(mkOp) ⇒ mkOp() :: prev
|
||||
case Ast.StageFactory(mkStage, _) ⇒ mkStage() :: prev
|
||||
|
||||
// Optimizations below
|
||||
case noMatch if !optimizations.fusion ⇒ prev
|
||||
|
|
@ -332,7 +327,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
|
|||
val (pub, value) = createSource(flowName)
|
||||
(value, attachSink(pub, flowName))
|
||||
} else {
|
||||
val id = processorForNode[In, Out](identityTransform, flowName, 1)
|
||||
val id = processorForNode[In, Out](identityStageNode, flowName, 1)
|
||||
(attachSource(id, flowName), attachSink(id, flowName))
|
||||
}
|
||||
} else {
|
||||
|
|
@ -344,7 +339,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
|
|||
new MaterializedPipe(source, sourceValue, sink, sinkValue)
|
||||
}
|
||||
//FIXME Should this be a dedicated AstNode?
|
||||
private[this] val identityTransform = Ast.Transform("identity", () ⇒ FlowOps.identityTransformer[Any])
|
||||
private[this] val identityStageNode = Ast.StageFactory(() ⇒ FlowOps.identityStage[Any], "identity")
|
||||
|
||||
def executionContext: ExecutionContext = dispatchers.lookup(settings.dispatcher match {
|
||||
case Deploy.NoDispatcherGiven ⇒ Dispatchers.DefaultDispatcherId
|
||||
|
|
@ -406,7 +401,7 @@ case class ActorBasedFlowMaterializer(override val settings: MaterializerSetting
|
|||
(List(subscriber), publishers)
|
||||
|
||||
case identity @ Ast.IdentityAstNode ⇒ // FIXME Why is IdentityAstNode a JunctionAStNode?
|
||||
val id = List(processorForNode[In, Out](identityTransform, identity.name, 1)) // FIXME is `identity.name` appropriate/unique here?
|
||||
val id = List(processorForNode[In, Out](identityStageNode, identity.name, 1)) // FIXME is `identity.name` appropriate/unique here?
|
||||
(id, id)
|
||||
}
|
||||
|
||||
|
|
@ -460,31 +455,26 @@ private[akka] object ActorProcessorFactory {
|
|||
def props(materializer: FlowMaterializer, op: AstNode): Props = {
|
||||
val settings = materializer.settings // USE THIS TO AVOID CLOSING OVER THE MATERIALIZER BELOW
|
||||
(op match {
|
||||
case Fused(ops, _) ⇒ Props(new ActorInterpreter(settings, ops))
|
||||
case Map(f) ⇒ Props(new ActorInterpreter(settings, List(fusing.Map(f))))
|
||||
case Filter(p) ⇒ Props(new ActorInterpreter(settings, List(fusing.Filter(p))))
|
||||
case Drop(n) ⇒ Props(new ActorInterpreter(settings, List(fusing.Drop(n))))
|
||||
case Take(n) ⇒ Props(new ActorInterpreter(settings, List(fusing.Take(n))))
|
||||
case Collect(pf) ⇒ Props(new ActorInterpreter(settings, List(fusing.Collect(pf))))
|
||||
case Scan(z, f) ⇒ Props(new ActorInterpreter(settings, List(fusing.Scan(z, f))))
|
||||
case Expand(s, f) ⇒ Props(new ActorInterpreter(settings, List(fusing.Expand(s, f))))
|
||||
case Conflate(s, f) ⇒ Props(new ActorInterpreter(settings, List(fusing.Conflate(s, f))))
|
||||
case Buffer(n, s) ⇒ Props(new ActorInterpreter(settings, List(fusing.Buffer(n, s))))
|
||||
case MapConcat(f) ⇒ Props(new ActorInterpreter(settings, List(fusing.MapConcat(f))))
|
||||
case Operate(mkOp) ⇒ Props(new ActorInterpreter(settings, List(mkOp())))
|
||||
case MapAsync(f) ⇒ Props(new MapAsyncProcessorImpl(settings, f))
|
||||
case MapAsyncUnordered(f) ⇒ Props(new MapAsyncUnorderedProcessorImpl(settings, f))
|
||||
case Grouped(n) ⇒ Props(new ActorInterpreter(settings, List(fusing.Grouped(n))))
|
||||
case GroupBy(f) ⇒ Props(new GroupByProcessorImpl(settings, f))
|
||||
case PrefixAndTail(n) ⇒ Props(new PrefixAndTailImpl(settings, n))
|
||||
case SplitWhen(p) ⇒ Props(new SplitWhenProcessorImpl(settings, p))
|
||||
case ConcatAll ⇒ Props(new ConcatAllImpl(materializer)) //FIXME closes over the materializer, is this good?
|
||||
case t: Transform ⇒
|
||||
val tr = t.mkTransformer()
|
||||
Props(new TransformProcessorImpl(settings, tr))
|
||||
case t: TimerTransform ⇒
|
||||
val tr = t.mkTransformer()
|
||||
Props(new TimerTransformerProcessorsImpl(settings, tr))
|
||||
case Fused(ops, _) ⇒ ActorInterpreter.props(settings, ops)
|
||||
case Map(f) ⇒ ActorInterpreter.props(settings, List(fusing.Map(f)))
|
||||
case Filter(p) ⇒ ActorInterpreter.props(settings, List(fusing.Filter(p)))
|
||||
case Drop(n) ⇒ ActorInterpreter.props(settings, List(fusing.Drop(n)))
|
||||
case Take(n) ⇒ ActorInterpreter.props(settings, List(fusing.Take(n)))
|
||||
case Collect(pf) ⇒ ActorInterpreter.props(settings, List(fusing.Collect(pf)))
|
||||
case Scan(z, f) ⇒ ActorInterpreter.props(settings, List(fusing.Scan(z, f)))
|
||||
case Expand(s, f) ⇒ ActorInterpreter.props(settings, List(fusing.Expand(s, f)))
|
||||
case Conflate(s, f) ⇒ ActorInterpreter.props(settings, List(fusing.Conflate(s, f)))
|
||||
case Buffer(n, s) ⇒ ActorInterpreter.props(settings, List(fusing.Buffer(n, s)))
|
||||
case MapConcat(f) ⇒ ActorInterpreter.props(settings, List(fusing.MapConcat(f)))
|
||||
case MapAsync(f) ⇒ MapAsyncProcessorImpl.props(settings, f)
|
||||
case MapAsyncUnordered(f) ⇒ MapAsyncUnorderedProcessorImpl.props(settings, f)
|
||||
case Grouped(n) ⇒ ActorInterpreter.props(settings, List(fusing.Grouped(n)))
|
||||
case GroupBy(f) ⇒ GroupByProcessorImpl.props(settings, f)
|
||||
case PrefixAndTail(n) ⇒ PrefixAndTailImpl.props(settings, n)
|
||||
case SplitWhen(p) ⇒ SplitWhenProcessorImpl.props(settings, p)
|
||||
case ConcatAll ⇒ ConcatAllImpl.props(materializer) //FIXME closes over the materializer, is this good?
|
||||
case StageFactory(mkStage, _) ⇒ ActorInterpreter.props(settings, List(mkStage()))
|
||||
case TimerTransform(mkStage, _) ⇒ TimerTransformerProcessorsImpl.props(settings, mkStage())
|
||||
}).withDispatcher(settings.dispatcher)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,6 +5,15 @@ package akka.stream.impl
|
|||
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.scaladsl.Sink
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object ConcatAllImpl {
|
||||
def props(materializer: FlowMaterializer): Props =
|
||||
Props(new ConcatAllImpl(materializer))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -5,6 +5,15 @@ package akka.stream.impl
|
|||
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object GroupByProcessorImpl {
|
||||
def props(settings: MaterializerSettings, keyFor: Any ⇒ Any): Props =
|
||||
Props(new GroupByProcessorImpl(settings, keyFor))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -11,12 +11,16 @@ import scala.util.control.NonFatal
|
|||
import akka.stream.MaterializerSettings
|
||||
import akka.pattern.pipe
|
||||
import scala.annotation.tailrec
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MapAsyncProcessorImpl {
|
||||
|
||||
def props(settings: MaterializerSettings, f: Any ⇒ Future[Any]): Props =
|
||||
Props(new MapAsyncProcessorImpl(settings, f))
|
||||
|
||||
object FutureElement {
|
||||
implicit val ordering: Ordering[FutureElement] = new Ordering[FutureElement] {
|
||||
def compare(a: FutureElement, b: FutureElement): Int = {
|
||||
|
|
|
|||
|
|
@ -8,11 +8,15 @@ import scala.util.control.NonFatal
|
|||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.pattern.pipe
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MapAsyncUnorderedProcessorImpl {
|
||||
def props(settings: MaterializerSettings, f: Any ⇒ Future[Any]): Props =
|
||||
Props(new MapAsyncUnorderedProcessorImpl(settings, f))
|
||||
|
||||
case class FutureElement(element: Any)
|
||||
case class FutureFailure(cause: Throwable)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,6 +6,15 @@ package akka.stream.impl
|
|||
import scala.collection.immutable
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object PrefixAndTailImpl {
|
||||
def props(settings: MaterializerSettings, takeMax: Int): Props =
|
||||
Props(new PrefixAndTailImpl(settings, takeMax))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -1,76 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.impl
|
||||
|
||||
import akka.actor.Props
|
||||
import akka.stream.{ MaterializerSettings, TransformerLike }
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class TransformProcessorImpl(_settings: MaterializerSettings, transformer: TransformerLike[Any, Any])
|
||||
extends ActorProcessorImpl(_settings) with Emit {
|
||||
|
||||
var errorEvent: Option[Throwable] = None
|
||||
|
||||
override def preStart(): Unit = {
|
||||
super.preStart()
|
||||
nextPhase(running)
|
||||
}
|
||||
|
||||
override def onError(e: Throwable): Unit = {
|
||||
try {
|
||||
transformer.onError(e)
|
||||
errorEvent = Some(e)
|
||||
pump()
|
||||
} catch { case NonFatal(ex) ⇒ fail(ex) }
|
||||
}
|
||||
|
||||
object NeedsInputAndDemandOrCompletion extends TransferState {
|
||||
def isReady = (primaryInputs.inputsAvailable && primaryOutputs.demandAvailable) || primaryInputs.inputsDepleted
|
||||
def isCompleted = primaryOutputs.isClosed
|
||||
}
|
||||
|
||||
private val runningPhase: TransferPhase = TransferPhase(NeedsInputAndDemandOrCompletion) { () ⇒
|
||||
if (primaryInputs.inputsDepleted) nextPhase(terminate)
|
||||
else {
|
||||
emits = transformer.onNext(primaryInputs.dequeueInputElement())
|
||||
if (transformer.isComplete) emitAndThen(terminate)
|
||||
else emitAndThen(running)
|
||||
}
|
||||
}
|
||||
|
||||
def running: TransferPhase = runningPhase
|
||||
|
||||
val terminate = TransferPhase(Always) { () ⇒
|
||||
emits = transformer.onTermination(errorEvent)
|
||||
emitAndThen(completedPhase)
|
||||
}
|
||||
|
||||
override def toString: String = s"Transformer(emits=$emits, transformer=$transformer)"
|
||||
|
||||
override def postStop(): Unit = try super.postStop() finally transformer.cleanup()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object IdentityProcessorImpl {
|
||||
def props(settings: MaterializerSettings): Props = Props(new IdentityProcessorImpl(settings))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class IdentityProcessorImpl(_settings: MaterializerSettings) extends ActorProcessorImpl(_settings) {
|
||||
|
||||
val running: TransferPhase = TransferPhase(primaryInputs.NeedsInput && primaryOutputs.NeedsDemand) { () ⇒
|
||||
primaryOutputs.enqueueOutputElement(primaryInputs.dequeueInputElement())
|
||||
}
|
||||
|
||||
nextPhase(running)
|
||||
}
|
||||
|
|
@ -5,6 +5,15 @@ package akka.stream.impl
|
|||
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.scaladsl.Source
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object SplitWhenProcessorImpl {
|
||||
def props(settings: MaterializerSettings, splitPredicate: Any ⇒ Boolean): Props =
|
||||
Props(new SplitWhenProcessorImpl(settings, splitPredicate))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -7,6 +7,12 @@ import java.util.LinkedList
|
|||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.TimerTransformer
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.Props
|
||||
|
||||
private[akka] object TimerTransformerProcessorsImpl {
|
||||
def props(settings: MaterializerSettings, transformer: TimerTransformer[Any, Any]): Props =
|
||||
Props(new TimerTransformerProcessorsImpl(settings, transformer))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -14,17 +20,29 @@ import scala.util.control.NonFatal
|
|||
private[akka] class TimerTransformerProcessorsImpl(
|
||||
_settings: MaterializerSettings,
|
||||
transformer: TimerTransformer[Any, Any])
|
||||
extends TransformProcessorImpl(_settings, transformer) {
|
||||
extends ActorProcessorImpl(_settings) with Emit {
|
||||
import TimerTransformer._
|
||||
|
||||
var errorEvent: Option[Throwable] = None
|
||||
|
||||
override def preStart(): Unit = {
|
||||
super.preStart()
|
||||
nextPhase(running)
|
||||
transformer.start(context)
|
||||
}
|
||||
|
||||
override def postStop(): Unit = {
|
||||
override def postStop(): Unit =
|
||||
try {
|
||||
super.postStop()
|
||||
transformer.stop()
|
||||
} finally transformer.cleanup()
|
||||
|
||||
override def onError(e: Throwable): Unit = {
|
||||
try {
|
||||
transformer.onError(e)
|
||||
errorEvent = Some(e)
|
||||
pump()
|
||||
} catch { case NonFatal(ex) ⇒ fail(ex) }
|
||||
}
|
||||
|
||||
val schedulerInputs: Inputs = new DefaultInputTransferStates {
|
||||
|
|
@ -58,7 +76,7 @@ private[akka] class TimerTransformerProcessorsImpl(
|
|||
def isCompleted = false
|
||||
}
|
||||
|
||||
private val runningPhase: TransferPhase = TransferPhase(RunningCondition) { () ⇒
|
||||
private val running: TransferPhase = TransferPhase(RunningCondition) { () ⇒
|
||||
if (primaryInputs.inputsDepleted || (transformer.isComplete && !schedulerInputs.inputsAvailable)) {
|
||||
nextPhase(terminate)
|
||||
} else if (schedulerInputs.inputsAvailable) {
|
||||
|
|
@ -71,6 +89,11 @@ private[akka] class TimerTransformerProcessorsImpl(
|
|||
}
|
||||
}
|
||||
|
||||
override def running: TransferPhase = runningPhase
|
||||
private val terminate = TransferPhase(Always) { () ⇒
|
||||
emits = transformer.onTermination(errorEvent)
|
||||
emitAndThen(completedPhase)
|
||||
}
|
||||
|
||||
override def toString: String = s"Transformer(emits=$emits, transformer=$transformer)"
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,21 +4,21 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import java.util.Arrays
|
||||
|
||||
import akka.actor.{ Actor, ActorRef }
|
||||
import akka.event.Logging
|
||||
import akka.stream.MaterializerSettings
|
||||
import akka.stream.actor.ActorSubscriber.OnSubscribe
|
||||
import akka.stream.actor.ActorSubscriberMessage.{ OnNext, OnError, OnComplete }
|
||||
import akka.stream.impl._
|
||||
import akka.stream.stage._
|
||||
import org.reactivestreams.{ Subscriber, Subscription }
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.Props
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class BatchingActorInputBoundary(val size: Int) extends BoundaryOp {
|
||||
private[akka] class BatchingActorInputBoundary(val size: Int) extends BoundaryStage {
|
||||
require(size > 0, "buffer size cannot be zero")
|
||||
require((size & (size - 1)) == 0, "buffer size must be a power of two")
|
||||
|
||||
|
|
@ -60,25 +60,25 @@ private[akka] class BatchingActorInputBoundary(val size: Int) extends BoundaryOp
|
|||
}
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): Directive =
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("BUG: Cannot push the upstream boundary")
|
||||
|
||||
override def onPull(ctxt: BoundaryContext): Directive = {
|
||||
if (inputBufferElements > 1) ctxt.push(dequeue())
|
||||
override def onPull(ctx: BoundaryContext): Directive = {
|
||||
if (inputBufferElements > 1) ctx.push(dequeue())
|
||||
else if (inputBufferElements == 1) {
|
||||
if (upstreamCompleted) ctxt.pushAndFinish(dequeue())
|
||||
else ctxt.push(dequeue())
|
||||
if (upstreamCompleted) ctx.pushAndFinish(dequeue())
|
||||
else ctx.push(dequeue())
|
||||
} else if (upstreamCompleted) {
|
||||
ctxt.finish()
|
||||
ctx.finish()
|
||||
} else {
|
||||
downstreamWaiting = true
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
}
|
||||
|
||||
override def onDownstreamFinish(ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
cancel()
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
def cancel(): Unit = {
|
||||
|
|
@ -143,7 +143,7 @@ private[akka] class BatchingActorInputBoundary(val size: Int) extends BoundaryOp
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorOutputBoundary(val actor: ActorRef) extends BoundaryOp {
|
||||
private[akka] class ActorOutputBoundary(val actor: ActorRef) extends BoundaryStage {
|
||||
|
||||
private var exposedPublisher: ActorPublisher[Any] = _
|
||||
|
||||
|
|
@ -177,27 +177,27 @@ private[akka] class ActorOutputBoundary(val actor: ActorRef) extends BoundaryOp
|
|||
}
|
||||
}
|
||||
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): Directive = {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
onNext(elem)
|
||||
if (downstreamDemand > 0) ctxt.pull()
|
||||
else if (downstreamCompleted) ctxt.finish()
|
||||
if (downstreamDemand > 0) ctx.pull()
|
||||
else if (downstreamCompleted) ctx.finish()
|
||||
else {
|
||||
upstreamWaiting = true
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
}
|
||||
|
||||
override def onPull(ctxt: BoundaryContext): Directive =
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("BUG: Cannot pull the downstream boundary")
|
||||
|
||||
override def onUpstreamFinish(ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
complete()
|
||||
ctxt.finish()
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onFailure(cause: Throwable, ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
fail(cause)
|
||||
ctxt.fail(cause)
|
||||
ctx.fail(cause)
|
||||
}
|
||||
|
||||
private def subscribePending(subscribers: Seq[Subscriber[Any]]): Unit =
|
||||
|
|
@ -245,7 +245,15 @@ private[akka] class ActorOutputBoundary(val actor: ActorRef) extends BoundaryOp
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorInterpreter(settings: MaterializerSettings, ops: Seq[Op[_, _, _, _, _]])
|
||||
private[akka] object ActorInterpreter {
|
||||
def props(settings: MaterializerSettings, ops: Seq[Stage[_, _]]): Props =
|
||||
Props(new ActorInterpreter(settings, ops))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ActorInterpreter(val settings: MaterializerSettings, val ops: Seq[Stage[_, _]])
|
||||
extends Actor {
|
||||
|
||||
private val upstream = new BatchingActorInputBoundary(settings.initialInputBufferSize)
|
||||
|
|
|
|||
|
|
@ -4,63 +4,38 @@
|
|||
package akka.stream.impl.fusing
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.breakOut
|
||||
import scala.util.control.NonFatal
|
||||
import akka.stream.stage._
|
||||
|
||||
// TODO:
|
||||
// fix jumpback table with keep-going-on-complete ops (we might jump between otherwise isolated execution regions)
|
||||
// implement grouped, buffer
|
||||
// add recover
|
||||
|
||||
trait Op[In, Out, PushD <: Directive, PullD <: Directive, Ctxt <: Context[Out]] {
|
||||
private[fusing] var holding = false
|
||||
private[fusing] var allowedToPush = false
|
||||
private[fusing] var terminationPending = false
|
||||
|
||||
def isHolding: Boolean = holding
|
||||
def isFinishing: Boolean = terminationPending
|
||||
def onPush(elem: In, ctxt: Ctxt): PushD
|
||||
def onPull(ctxt: Ctxt): PullD
|
||||
def onUpstreamFinish(ctxt: Ctxt): TerminationDirective = ctxt.finish()
|
||||
def onDownstreamFinish(ctxt: Ctxt): TerminationDirective = ctxt.finish()
|
||||
def onFailure(cause: Throwable, ctxt: Ctxt): TerminationDirective = ctxt.fail(cause)
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* `BoundaryStage` implementations are meant to communicate with the external world. These stages do not have most of the
|
||||
* safety properties enforced and should be used carefully. One important ability of BoundaryStages that they can take
|
||||
* off an execution signal by calling `ctx.exit()`. This is typically used immediately after an external signal has
|
||||
* been produced (for example an actor message). BoundaryStages can also kickstart execution by calling `enter()` which
|
||||
* returns a context they can use to inject signals into the interpreter. There is no checks in place to enforce that
|
||||
* the number of signals taken out by exit() and the number of signals returned via enter() are the same -- using this
|
||||
* stage type needs extra care from the implementer.
|
||||
*
|
||||
* BoundaryStages are the elements that make the interpreter *tick*, there is no other way to start the interpreter
|
||||
* than using a BoundaryStage.
|
||||
*/
|
||||
private[akka] abstract class BoundaryStage extends AbstractStage[Any, Any, Directive, Directive, BoundaryContext] {
|
||||
private[fusing] var bctx: BoundaryContext = _
|
||||
def enter(): BoundaryContext = bctx
|
||||
}
|
||||
|
||||
trait DeterministicOp[In, Out] extends Op[In, Out, Directive, Directive, Context[Out]]
|
||||
trait DetachedOp[In, Out] extends Op[In, Out, UpstreamDirective, DownstreamDirective, DetachedContext[Out]]
|
||||
trait BoundaryOp extends Op[Any, Any, Directive, Directive, BoundaryContext] {
|
||||
private[fusing] var bctxt: BoundaryContext = _
|
||||
def enter(): BoundaryContext = bctxt
|
||||
}
|
||||
|
||||
trait TransitivePullOp[In, Out] extends DeterministicOp[In, Out] {
|
||||
final override def onPull(ctxt: Context[Out]): Directive = ctxt.pull()
|
||||
}
|
||||
|
||||
sealed trait Directive
|
||||
sealed trait UpstreamDirective extends Directive
|
||||
sealed trait DownstreamDirective extends Directive
|
||||
sealed trait TerminationDirective extends Directive
|
||||
final class FreeDirective extends UpstreamDirective with DownstreamDirective with TerminationDirective
|
||||
|
||||
sealed trait Context[Out] {
|
||||
def push(elem: Out): DownstreamDirective
|
||||
def pull(): UpstreamDirective
|
||||
def finish(): FreeDirective
|
||||
def pushAndFinish(elem: Out): DownstreamDirective
|
||||
def fail(cause: Throwable): FreeDirective
|
||||
def absorbTermination(): TerminationDirective
|
||||
}
|
||||
|
||||
trait DetachedContext[Out] extends Context[Out] {
|
||||
def hold(): FreeDirective
|
||||
def pushAndPull(elem: Out): FreeDirective
|
||||
}
|
||||
|
||||
trait BoundaryContext extends Context[Any] {
|
||||
def exit(): FreeDirective
|
||||
}
|
||||
|
||||
object OneBoundedInterpreter {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object OneBoundedInterpreter {
|
||||
final val PhantomDirective = null
|
||||
|
||||
/**
|
||||
|
|
@ -70,16 +45,18 @@ object OneBoundedInterpreter {
|
|||
* paths again. When finishing an op this op is injected in its place to isolate upstream and downstream execution
|
||||
* domains.
|
||||
*/
|
||||
private[akka] object Finished extends BoundaryOp {
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): UpstreamDirective = ctxt.finish()
|
||||
override def onPull(ctxt: BoundaryContext): DownstreamDirective = ctxt.finish()
|
||||
override def onUpstreamFinish(ctxt: BoundaryContext): TerminationDirective = ctxt.exit()
|
||||
override def onDownstreamFinish(ctxt: BoundaryContext): TerminationDirective = ctxt.exit()
|
||||
override def onFailure(cause: Throwable, ctxt: BoundaryContext): TerminationDirective = ctxt.exit()
|
||||
private[akka] object Finished extends BoundaryStage {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): UpstreamDirective = ctx.finish()
|
||||
override def onPull(ctx: BoundaryContext): DownstreamDirective = ctx.finish()
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
override def onDownstreamFinish(ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = ctx.exit()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* One-bounded interpreter for a linear chain of stream operations (graph support is possible and will be implemented
|
||||
* later)
|
||||
*
|
||||
|
|
@ -105,70 +82,70 @@ object OneBoundedInterpreter {
|
|||
* time. This "exactly one" property is enforced by proper types and runtime checks where needed. Currently there are
|
||||
* three kinds of ops:
|
||||
*
|
||||
* - DeterministicOp implementations participate in 1-bounded regions. For every external non-completion signal these
|
||||
* - PushPullStage implementations participate in 1-bounded regions. For every external non-completion signal these
|
||||
* ops produce *exactly one* signal (completion is different, explained later) therefore keeping the number of events
|
||||
* the same: exactly one.
|
||||
*
|
||||
* - DetachedOp implementations are boundaries between 1-bounded regions. This means that they need to enforce the
|
||||
* "exactly one" property both on their upstream and downstream regions. As a consequence a DetachedOp can never
|
||||
* answer an onPull with a ctxt.pull() or answer an onPush() with a ctxt.push() since such an action would "steal"
|
||||
* - DetachedStage implementations are boundaries between 1-bounded regions. This means that they need to enforce the
|
||||
* "exactly one" property both on their upstream and downstream regions. As a consequence a DetachedStage can never
|
||||
* answer an onPull with a ctx.pull() or answer an onPush() with a ctx.push() since such an action would "steal"
|
||||
* the event from one region (resulting in zero signals) and would inject it to the other region (resulting in two
|
||||
* signals). However DetachedOps have the ability to call ctxt.hold() as a response to onPush/onPull which temporarily
|
||||
* signals). However DetachedStages have the ability to call ctx.hold() as a response to onPush/onPull which temporarily
|
||||
* takes the signal off and stops execution, at the same time putting the op in a "holding" state. If the op is in a
|
||||
* holding state it contains one absorbed signal, therefore in this state the only possible command to call is
|
||||
* ctxt.pushAndPull() which results in two events making the balance right again:
|
||||
* ctx.pushAndPull() which results in two events making the balance right again:
|
||||
* 1 hold + 1 external event = 2 external event
|
||||
* This mechanism allows synchronization between the upstream and downstream regions which otherwise can progress
|
||||
* independently.
|
||||
*
|
||||
* - BoundaryOp implementations are meant to communicate with the external world. These ops do not have most of the
|
||||
* safety properties enforced and should be used carefully. One important ability of BoundaryOps that they can take
|
||||
* off an execution signal by calling ctxt.exit(). This is typically used immediately after an external signal has
|
||||
* been produced (for example an actor message). BoundaryOps can also kickstart execution by calling enter() which
|
||||
* - BoundaryStage implementations are meant to communicate with the external world. These ops do not have most of the
|
||||
* safety properties enforced and should be used carefully. One important ability of BoundaryStages that they can take
|
||||
* off an execution signal by calling ctx.exit(). This is typically used immediately after an external signal has
|
||||
* been produced (for example an actor message). BoundaryStages can also kickstart execution by calling enter() which
|
||||
* returns a context they can use to inject signals into the interpreter. There is no checks in place to enforce that
|
||||
* the number of signals taken out by exit() and the number of signals returned via enter() are the same -- using this
|
||||
* op type needs extra care from the implementer.
|
||||
* BoundaryOps are the elements that make the interpreter *tick*, there is no other way to start the interpreter
|
||||
* than using a BoundaryOp.
|
||||
* BoundaryStages are the elements that make the interpreter *tick*, there is no other way to start the interpreter
|
||||
* than using a BoundaryStage.
|
||||
*
|
||||
* Operations are allowed to do early completion and cancel/complete their upstreams and downstreams. It is *not*
|
||||
* allowed however to do these independently to avoid isolated execution islands. The only call possible is ctxt.finish()
|
||||
* allowed however to do these independently to avoid isolated execution islands. The only call possible is ctx.finish()
|
||||
* which is a combination of cancel/complete.
|
||||
* Since onComplete is not a backpressured signal it is sometimes preferable to push a final element and then immediately
|
||||
* finish. This combination is exposed as pushAndFinish() which enables op writers to propagate completion events without
|
||||
* waiting for an extra round of pull.
|
||||
* Another peculiarity is how to convert termination events (complete/failure) into elements. The problem
|
||||
* here is that the termination events are not backpressured while elements are. This means that simply calling ctxt.push()
|
||||
* here is that the termination events are not backpressured while elements are. This means that simply calling ctx.push()
|
||||
* as a response to onUpstreamFinished() will very likely break boundedness and result in a buffer overflow somewhere.
|
||||
* Therefore the only allowed command in this case is ctxt.absorbTermination() which stops the propagation of the
|
||||
* Therefore the only allowed command in this case is ctx.absorbTermination() which stops the propagation of the
|
||||
* termination signal, and puts the op in a finishing state. Depending on whether the op has a pending pull signal it has
|
||||
* not yet "consumed" by a push its onPull() handler might be called immediately.
|
||||
*
|
||||
* In order to execute different individual execution regions the interpreter uses the callstack to schedule these. The
|
||||
* current execution forking operations are
|
||||
* - ctxt.finish() which starts a wave of completion and cancellation in two directions. When an op calls finish()
|
||||
* - ctx.finish() which starts a wave of completion and cancellation in two directions. When an op calls finish()
|
||||
* it is immediately replaced by an artificial Finished op which makes sure that the two execution paths are isolated
|
||||
* forever.
|
||||
* - ctxt.fail() which is similar to finish()
|
||||
* - ctxt.pushAndPull() which (as a response to a previous ctxt.hold()) starts a wawe of downstream push and upstream
|
||||
* - ctx.fail() which is similar to finish()
|
||||
* - ctx.pushAndPull() which (as a response to a previous ctx.hold()) starts a wawe of downstream push and upstream
|
||||
* pull. The two execution paths are isolated by the op itself since onPull() from downstream can only answered by hold or
|
||||
* push, while onPush() from upstream can only answered by hold or pull -- it is impossible to "cross" the op.
|
||||
* - ctxt.pushAndFinish() which is different from the forking ops above because the execution of push and finish happens on
|
||||
* - ctx.pushAndFinish() which is different from the forking ops above because the execution of push and finish happens on
|
||||
* the same execution region and they are order dependent, too.
|
||||
* The interpreter tracks the depth of recursive forking and allows various strategies of dealing with the situation
|
||||
* when this depth reaches a certain limit. In the simplest case an error is reported (this is very useful for stress
|
||||
* testing and finding callstack wasting bugs), in the other case the forked call is scheduled via a list -- i.e. instead
|
||||
* of the stack the heap is used.
|
||||
*/
|
||||
class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 100, val overflowToHeap: Boolean = true) {
|
||||
private[akka] class OneBoundedInterpreter(ops: Seq[Stage[_, _]], val forkLimit: Int = 100, val overflowToHeap: Boolean = true) {
|
||||
import OneBoundedInterpreter._
|
||||
type UntypedOp = Op[Any, Any, Directive, Directive, DetachedContext[Any]]
|
||||
type UntypedOp = AbstractStage[Any, Any, Directive, Directive, Context[Any]]
|
||||
require(ops.nonEmpty, "OneBoundedInterpreter cannot be created without at least one Op")
|
||||
|
||||
private val pipeline = ops.toArray.asInstanceOf[Array[UntypedOp]]
|
||||
private val pipeline: Array[UntypedOp] = ops.map(_.asInstanceOf[UntypedOp])(breakOut)
|
||||
|
||||
/**
|
||||
* This table is used to accelerate demand propagation upstream. All ops that implement TransitivePullOp are guaranteed
|
||||
* This table is used to accelerate demand propagation upstream. All ops that implement PushStage are guaranteed
|
||||
* to only do upstream propagation of demand signals, therefore it is not necessary to execute them but enough to
|
||||
* "jump over" them. This means that when a chain of one million maps gets a downstream demand it is propagated
|
||||
* to the upstream *in one step* instead of one million onPull() calls.
|
||||
|
|
@ -199,7 +176,7 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
var nextJumpBack = -1
|
||||
for (pos ← 0 until pipeline.length) {
|
||||
table(pos) = nextJumpBack
|
||||
if (!pipeline(pos).isInstanceOf[TransitivePullOp[_, _]]) nextJumpBack = pos
|
||||
if (!pipeline(pos).isInstanceOf[PushStage[_, _]]) nextJumpBack = pos
|
||||
}
|
||||
table
|
||||
}
|
||||
|
|
@ -217,7 +194,7 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
|
||||
override def pull(): UpstreamDirective = {
|
||||
if (pipeline(activeOp).holding) throw new IllegalStateException("Cannot pull while holding, only pushAndPull")
|
||||
pipeline(activeOp).allowedToPush = !pipeline(activeOp).isInstanceOf[DetachedOp[_, _]]
|
||||
pipeline(activeOp).allowedToPush = !pipeline(activeOp).isInstanceOf[DetachedStage[_, _]]
|
||||
state = Pulling
|
||||
PhantomDirective
|
||||
}
|
||||
|
|
@ -228,6 +205,8 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
PhantomDirective
|
||||
}
|
||||
|
||||
def isFinishing: Boolean = pipeline(activeOp).terminationPending
|
||||
|
||||
override def pushAndFinish(elem: Any): DownstreamDirective = {
|
||||
pipeline(activeOp) = Finished.asInstanceOf[UntypedOp]
|
||||
// This MUST be an unsafeFork because the execution of PushFinish MUST strictly come before the finish execution
|
||||
|
|
@ -253,6 +232,8 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
exit()
|
||||
}
|
||||
|
||||
override def isHolding: Boolean = pipeline(activeOp).holding
|
||||
|
||||
override def pushAndPull(elem: Any): FreeDirective = {
|
||||
if (!pipeline(activeOp).holding) throw new IllegalStateException("Cannot pushAndPull without holding first")
|
||||
pipeline(activeOp).holding = false
|
||||
|
|
@ -276,14 +257,14 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
private object Pushing extends State {
|
||||
override def advance(): Unit = {
|
||||
activeOp += 1
|
||||
pipeline(activeOp).onPush(elementInFlight, ctxt = this)
|
||||
pipeline(activeOp).onPush(elementInFlight, ctx = this)
|
||||
}
|
||||
}
|
||||
|
||||
private object PushFinish extends State {
|
||||
override def advance(): Unit = {
|
||||
activeOp += 1
|
||||
pipeline(activeOp).onPush(elementInFlight, ctxt = this)
|
||||
pipeline(activeOp).onPush(elementInFlight, ctx = this)
|
||||
}
|
||||
|
||||
override def pushAndFinish(elem: Any): DownstreamDirective = {
|
||||
|
|
@ -302,7 +283,7 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
override def advance(): Unit = {
|
||||
elementInFlight = null
|
||||
activeOp = jumpBacks(activeOp)
|
||||
pipeline(activeOp).onPull(ctxt = this)
|
||||
pipeline(activeOp).onPull(ctx = this)
|
||||
}
|
||||
|
||||
override def hold(): FreeDirective = {
|
||||
|
|
@ -317,7 +298,9 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
elementInFlight = null
|
||||
pipeline(activeOp) = Finished.asInstanceOf[UntypedOp]
|
||||
activeOp += 1
|
||||
if (!pipeline(activeOp).isFinishing) pipeline(activeOp).onUpstreamFinish(ctxt = this)
|
||||
|
||||
// FIXME issue #16345, ArrayIndexOutOfBoundsException
|
||||
if (!pipeline(activeOp).terminationPending) pipeline(activeOp).onUpstreamFinish(ctx = this)
|
||||
else exit()
|
||||
}
|
||||
|
||||
|
|
@ -330,7 +313,7 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
pipeline(activeOp).terminationPending = true
|
||||
pipeline(activeOp).holding = false
|
||||
// FIXME: This state is potentially corrupted by the jumpBackTable (not updated when jumping over)
|
||||
if (pipeline(activeOp).allowedToPush) pipeline(activeOp).onPull(ctxt = Pulling)
|
||||
if (pipeline(activeOp).allowedToPush) pipeline(activeOp).onPull(ctx = Pulling)
|
||||
else exit()
|
||||
PhantomDirective
|
||||
}
|
||||
|
|
@ -341,7 +324,9 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
elementInFlight = null
|
||||
pipeline(activeOp) = Finished.asInstanceOf[UntypedOp]
|
||||
activeOp -= 1
|
||||
if (!pipeline(activeOp).isFinishing) pipeline(activeOp).onDownstreamFinish(ctxt = this)
|
||||
|
||||
// FIXME issue #16345, ArrayIndexOutOfBoundsException
|
||||
if (!pipeline(activeOp).terminationPending) pipeline(activeOp).onDownstreamFinish(ctx = this)
|
||||
else exit()
|
||||
}
|
||||
|
||||
|
|
@ -356,13 +341,13 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
elementInFlight = null
|
||||
pipeline(activeOp) = Finished.asInstanceOf[UntypedOp]
|
||||
activeOp += 1
|
||||
pipeline(activeOp).onFailure(cause, ctxt = this)
|
||||
pipeline(activeOp).onUpstreamFailure(cause, ctx = this)
|
||||
}
|
||||
|
||||
override def absorbTermination(): TerminationDirective = {
|
||||
pipeline(activeOp).terminationPending = true
|
||||
pipeline(activeOp).holding = false
|
||||
if (pipeline(activeOp).allowedToPush) pipeline(activeOp).onPull(ctxt = Pulling)
|
||||
if (pipeline(activeOp).allowedToPush) pipeline(activeOp).onPull(ctx = Pulling)
|
||||
else exit()
|
||||
PhantomDirective
|
||||
}
|
||||
|
|
@ -421,7 +406,6 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
state = forkState
|
||||
execute()
|
||||
activeOp = savePos
|
||||
PhantomDirective
|
||||
}
|
||||
|
||||
def init(): Unit = {
|
||||
|
|
@ -432,13 +416,15 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
def isFinished: Boolean = pipeline(Upstream) == Finished && pipeline(Downstream) == Finished
|
||||
|
||||
/**
|
||||
* This method injects a Context to each of the BoundaryOps. This will be the context returned by enter().
|
||||
* This method injects a Context to each of the BoundaryStages. This will be the context returned by enter().
|
||||
*/
|
||||
private def initBoundaries(): Unit = {
|
||||
var op = 0
|
||||
while (op < pipeline.length) {
|
||||
if (pipeline(op).isInstanceOf[BoundaryOp]) {
|
||||
pipeline(op).asInstanceOf[BoundaryOp].bctxt = new State {
|
||||
// FIXME try to change this to a pattern match `case boundary: BoundaryStage`
|
||||
// but that doesn't work with current Context types
|
||||
if (pipeline(op).isInstanceOf[BoundaryStage]) {
|
||||
pipeline(op).asInstanceOf[BoundaryStage].bctx = new State {
|
||||
val entryPoint = op
|
||||
|
||||
override def advance(): Unit = ()
|
||||
|
|
@ -499,7 +485,7 @@ class OneBoundedInterpreter(ops: Seq[Op[_, _, _, _, _]], val forkLimit: Int = 10
|
|||
private def runDetached(): Unit = {
|
||||
var op = pipeline.length - 1
|
||||
while (op >= 0) {
|
||||
if (pipeline(op).isInstanceOf[DetachedOp[_, _]]) {
|
||||
if (pipeline(op).isInstanceOf[DetachedStage[_, _]]) {
|
||||
activeOp = op
|
||||
state = Pulling
|
||||
execute()
|
||||
|
|
|
|||
|
|
@ -3,49 +3,57 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
object IteratorInterpreter {
|
||||
case class IteratorUpstream[T](input: Iterator[T]) extends DeterministicOp[T, T] {
|
||||
import akka.stream.stage._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object IteratorInterpreter {
|
||||
final case class IteratorUpstream[T](input: Iterator[T]) extends PushPullStage[T, T] {
|
||||
private var hasNext = input.hasNext
|
||||
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive =
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive =
|
||||
throw new UnsupportedOperationException("IteratorUpstream operates as a source, it cannot be pushed")
|
||||
|
||||
override def onPull(ctxt: Context[T]): Directive = {
|
||||
if (!hasNext) ctxt.finish()
|
||||
override def onPull(ctx: Context[T]): Directive = {
|
||||
if (!hasNext) ctx.finish()
|
||||
else {
|
||||
val elem = input.next()
|
||||
hasNext = input.hasNext
|
||||
if (!hasNext) ctxt.pushAndFinish(elem)
|
||||
else ctxt.push(elem)
|
||||
if (!hasNext) ctx.pushAndFinish(elem)
|
||||
else ctx.push(elem)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// don't let toString consume the iterator
|
||||
override def toString: String = "IteratorUpstream"
|
||||
}
|
||||
|
||||
case class IteratorDownstream[T]() extends BoundaryOp with Iterator[T] {
|
||||
final case class IteratorDownstream[T]() extends BoundaryStage with Iterator[T] {
|
||||
private var done = false
|
||||
private var nextElem: T = _
|
||||
private var needsPull = true
|
||||
private var lastError: Throwable = null
|
||||
|
||||
override def onPush(elem: Any, ctxt: BoundaryContext): Directive = {
|
||||
override def onPush(elem: Any, ctx: BoundaryContext): Directive = {
|
||||
nextElem = elem.asInstanceOf[T]
|
||||
needsPull = false
|
||||
ctxt.exit()
|
||||
ctx.exit()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: BoundaryContext): Directive =
|
||||
override def onPull(ctx: BoundaryContext): Directive =
|
||||
throw new UnsupportedOperationException("IteratorDownstream operates as a sink, it cannot be pulled")
|
||||
|
||||
override def onUpstreamFinish(ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFinish(ctx: BoundaryContext): TerminationDirective = {
|
||||
done = true
|
||||
ctxt.finish()
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
override def onFailure(cause: Throwable, ctxt: BoundaryContext): TerminationDirective = {
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: BoundaryContext): TerminationDirective = {
|
||||
done = true
|
||||
lastError = cause
|
||||
ctxt.finish()
|
||||
ctx.finish()
|
||||
}
|
||||
|
||||
private def pullIfNeeded(): Unit = {
|
||||
|
|
@ -56,27 +64,37 @@ object IteratorInterpreter {
|
|||
|
||||
override def hasNext: Boolean = {
|
||||
if (!done) pullIfNeeded()
|
||||
!(done && needsPull)
|
||||
!(done && needsPull) || (lastError ne null)
|
||||
}
|
||||
|
||||
override def next(): T = {
|
||||
if (!hasNext) {
|
||||
if (lastError != null) throw lastError
|
||||
else Iterator.empty.next()
|
||||
}
|
||||
if (lastError ne null) {
|
||||
val e = lastError
|
||||
lastError = null
|
||||
throw e
|
||||
} else if (!hasNext)
|
||||
Iterator.empty.next()
|
||||
else {
|
||||
needsPull = true
|
||||
nextElem
|
||||
}
|
||||
}
|
||||
|
||||
// don't let toString consume the iterator
|
||||
override def toString: String = "IteratorDownstream"
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
class IteratorInterpreter[I, O](val input: Iterator[I], val ops: Seq[DeterministicOp[_, _]]) {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class IteratorInterpreter[I, O](val input: Iterator[I], val ops: Seq[PushPullStage[_, _]]) {
|
||||
import akka.stream.impl.fusing.IteratorInterpreter._
|
||||
|
||||
private val upstream = IteratorUpstream(input)
|
||||
private val downstream = IteratorDownstream[O]()
|
||||
private val interpreter = new OneBoundedInterpreter(upstream +: ops.asInstanceOf[Seq[Op[_, _, _, _, _]]] :+ downstream)
|
||||
private val interpreter = new OneBoundedInterpreter(upstream +: ops.asInstanceOf[Seq[Stage[_, _]]] :+ downstream)
|
||||
interpreter.init()
|
||||
|
||||
def iterator: Iterator[O] = downstream
|
||||
|
|
|
|||
|
|
@ -3,25 +3,25 @@
|
|||
*/
|
||||
package akka.stream.impl.fusing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.stream.OverflowStrategy
|
||||
import akka.stream.impl.FixedSizeBuffer
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.stream.stage._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Map[In, Out](f: In ⇒ Out) extends TransitivePullOp[In, Out] {
|
||||
override def onPush(elem: In, ctxt: Context[Out]): Directive = ctxt.push(f(elem))
|
||||
private[akka] final case class Map[In, Out](f: In ⇒ Out) extends PushStage[In, Out] {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = ctx.push(f(elem))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Filter[T](p: T ⇒ Boolean) extends TransitivePullOp[T, T] {
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive =
|
||||
if (p(elem)) ctxt.push(elem)
|
||||
else ctxt.pull()
|
||||
private[akka] final case class Filter[T](p: T ⇒ Boolean) extends PushStage[T, T] {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive =
|
||||
if (p(elem)) ctx.push(elem)
|
||||
else ctx.pull()
|
||||
}
|
||||
|
||||
private[akka] final object Collect {
|
||||
|
|
@ -31,103 +31,103 @@ private[akka] final object Collect {
|
|||
final val NotApplied: Any ⇒ Any = _ ⇒ Collect.NotApplied
|
||||
}
|
||||
|
||||
private[akka] final case class Collect[In, Out](pf: PartialFunction[In, Out]) extends TransitivePullOp[In, Out] {
|
||||
private[akka] final case class Collect[In, Out](pf: PartialFunction[In, Out]) extends PushStage[In, Out] {
|
||||
import Collect.NotApplied
|
||||
override def onPush(elem: In, ctxt: Context[Out]): Directive =
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive =
|
||||
pf.applyOrElse(elem, NotApplied) match {
|
||||
case NotApplied ⇒ ctxt.pull()
|
||||
case result: Out ⇒ ctxt.push(result)
|
||||
case NotApplied ⇒ ctx.pull()
|
||||
case result: Out ⇒ ctx.push(result)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class MapConcat[In, Out](f: In ⇒ immutable.Seq[Out]) extends DeterministicOp[In, Out] {
|
||||
private[akka] final case class MapConcat[In, Out](f: In ⇒ immutable.Seq[Out]) extends PushPullStage[In, Out] {
|
||||
private var currentIterator: Iterator[Out] = Iterator.empty
|
||||
|
||||
override def onPush(elem: In, ctxt: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
currentIterator = f(elem).iterator
|
||||
if (currentIterator.isEmpty) ctxt.pull()
|
||||
else ctxt.push(currentIterator.next())
|
||||
if (currentIterator.isEmpty) ctx.pull()
|
||||
else ctx.push(currentIterator.next())
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[Out]): Directive =
|
||||
if (currentIterator.hasNext) ctxt.push(currentIterator.next())
|
||||
else if (isFinishing) ctxt.finish()
|
||||
else ctxt.pull()
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
if (currentIterator.hasNext) ctx.push(currentIterator.next())
|
||||
else if (ctx.isFinishing) ctx.finish()
|
||||
else ctx.pull()
|
||||
|
||||
override def onUpstreamFinish(ctxt: Context[Out]): TerminationDirective =
|
||||
ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: Context[Out]): TerminationDirective =
|
||||
ctx.absorbTermination()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Take[T](count: Int) extends TransitivePullOp[T, T] {
|
||||
private[akka] final case class Take[T](count: Int) extends PushStage[T, T] {
|
||||
private var left: Int = count
|
||||
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = {
|
||||
left -= 1
|
||||
if (left > 0) ctxt.push(elem)
|
||||
else if (left == 0) ctxt.pushAndFinish(elem)
|
||||
else ctxt.finish() //Handle negative take counts
|
||||
if (left > 0) ctx.push(elem)
|
||||
else if (left == 0) ctx.pushAndFinish(elem)
|
||||
else ctx.finish() //Handle negative take counts
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Drop[T](count: Int) extends TransitivePullOp[T, T] {
|
||||
private[akka] final case class Drop[T](count: Int) extends PushStage[T, T] {
|
||||
private var left: Int = count
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive =
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive =
|
||||
if (left > 0) {
|
||||
left -= 1
|
||||
ctxt.pull()
|
||||
} else ctxt.push(elem)
|
||||
ctx.pull()
|
||||
} else ctx.push(elem)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends DeterministicOp[In, Out] {
|
||||
private[akka] final case class Scan[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends PushPullStage[In, Out] {
|
||||
private var aggregator = zero
|
||||
|
||||
override def onPush(elem: In, ctxt: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
val old = aggregator
|
||||
aggregator = f(old, elem)
|
||||
ctxt.push(old)
|
||||
ctx.push(old)
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[Out]): Directive =
|
||||
if (isFinishing) ctxt.pushAndFinish(aggregator)
|
||||
else ctxt.pull()
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(aggregator)
|
||||
else ctx.pull()
|
||||
|
||||
override def onUpstreamFinish(ctxt: Context[Out]): TerminationDirective = ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: Context[Out]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends DeterministicOp[In, Out] {
|
||||
private[akka] final case class Fold[In, Out](zero: Out, f: (Out, In) ⇒ Out) extends PushPullStage[In, Out] {
|
||||
private var aggregator = zero
|
||||
|
||||
override def onPush(elem: In, ctxt: Context[Out]): Directive = {
|
||||
override def onPush(elem: In, ctx: Context[Out]): Directive = {
|
||||
aggregator = f(aggregator, elem)
|
||||
ctxt.pull()
|
||||
ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[Out]): Directive =
|
||||
if (isFinishing) ctxt.pushAndFinish(aggregator)
|
||||
else ctxt.pull()
|
||||
override def onPull(ctx: Context[Out]): Directive =
|
||||
if (ctx.isFinishing) ctx.pushAndFinish(aggregator)
|
||||
else ctx.pull()
|
||||
|
||||
override def onUpstreamFinish(ctxt: Context[Out]): TerminationDirective = ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: Context[Out]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Grouped[T](n: Int) extends DeterministicOp[T, immutable.Seq[T]] {
|
||||
private[akka] final case class Grouped[T](n: Int) extends PushPullStage[T, immutable.Seq[T]] {
|
||||
private val buf = {
|
||||
val b = Vector.newBuilder[T]
|
||||
b.sizeHint(n)
|
||||
|
|
@ -135,83 +135,83 @@ private[akka] final case class Grouped[T](n: Int) extends DeterministicOp[T, imm
|
|||
}
|
||||
private var left = n
|
||||
|
||||
override def onPush(elem: T, ctxt: Context[immutable.Seq[T]]): Directive = {
|
||||
override def onPush(elem: T, ctx: Context[immutable.Seq[T]]): Directive = {
|
||||
buf += elem
|
||||
left -= 1
|
||||
if (left == 0) {
|
||||
val emit = buf.result()
|
||||
buf.clear()
|
||||
left = n
|
||||
ctxt.push(emit)
|
||||
} else ctxt.pull()
|
||||
ctx.push(emit)
|
||||
} else ctx.pull()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: Context[immutable.Seq[T]]): Directive =
|
||||
if (isFinishing) {
|
||||
override def onPull(ctx: Context[immutable.Seq[T]]): Directive =
|
||||
if (ctx.isFinishing) {
|
||||
val elem = buf.result()
|
||||
buf.clear() //FIXME null out the reference to the `buf`?
|
||||
left = n
|
||||
ctxt.pushAndFinish(elem)
|
||||
} else ctxt.pull()
|
||||
ctx.pushAndFinish(elem)
|
||||
} else ctx.pull()
|
||||
|
||||
override def onUpstreamFinish(ctxt: Context[immutable.Seq[T]]): TerminationDirective =
|
||||
if (left == n) ctxt.finish()
|
||||
else ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: Context[immutable.Seq[T]]): TerminationDirective =
|
||||
if (left == n) ctx.finish()
|
||||
else ctx.absorbTermination()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extends DetachedOp[T, T] {
|
||||
private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowStrategy) extends DetachedStage[T, T] {
|
||||
import OverflowStrategy._
|
||||
|
||||
private val buffer = FixedSizeBuffer(size)
|
||||
|
||||
override def onPush(elem: T, ctxt: DetachedContext[T]): UpstreamDirective =
|
||||
if (isHolding) ctxt.pushAndPull(elem)
|
||||
else enqueueAction(ctxt, elem)
|
||||
override def onPush(elem: T, ctx: DetachedContext[T]): UpstreamDirective =
|
||||
if (ctx.isHolding) ctx.pushAndPull(elem)
|
||||
else enqueueAction(ctx, elem)
|
||||
|
||||
override def onPull(ctxt: DetachedContext[T]): DownstreamDirective = {
|
||||
if (isFinishing) {
|
||||
override def onPull(ctx: DetachedContext[T]): DownstreamDirective = {
|
||||
if (ctx.isFinishing) {
|
||||
val elem = buffer.dequeue().asInstanceOf[T]
|
||||
if (buffer.isEmpty) ctxt.pushAndFinish(elem)
|
||||
else ctxt.push(elem)
|
||||
} else if (isHolding) ctxt.pushAndPull(buffer.dequeue().asInstanceOf[T])
|
||||
else if (buffer.isEmpty) ctxt.hold()
|
||||
else ctxt.push(buffer.dequeue().asInstanceOf[T])
|
||||
if (buffer.isEmpty) ctx.pushAndFinish(elem)
|
||||
else ctx.push(elem)
|
||||
} else if (ctx.isHolding) ctx.pushAndPull(buffer.dequeue().asInstanceOf[T])
|
||||
else if (buffer.isEmpty) ctx.hold()
|
||||
else ctx.push(buffer.dequeue().asInstanceOf[T])
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctxt: DetachedContext[T]): TerminationDirective =
|
||||
if (buffer.isEmpty) ctxt.finish()
|
||||
else ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: DetachedContext[T]): TerminationDirective =
|
||||
if (buffer.isEmpty) ctx.finish()
|
||||
else ctx.absorbTermination()
|
||||
|
||||
val enqueueAction: (DetachedContext[T], T) ⇒ UpstreamDirective = {
|
||||
overflowStrategy match {
|
||||
case DropHead ⇒ { (ctxt, elem) ⇒
|
||||
case DropHead ⇒ { (ctx, elem) ⇒
|
||||
if (buffer.isFull) buffer.dropHead()
|
||||
buffer.enqueue(elem)
|
||||
ctxt.pull()
|
||||
ctx.pull()
|
||||
}
|
||||
case DropTail ⇒ { (ctxt, elem) ⇒
|
||||
case DropTail ⇒ { (ctx, elem) ⇒
|
||||
if (buffer.isFull) buffer.dropTail()
|
||||
buffer.enqueue(elem)
|
||||
ctxt.pull()
|
||||
ctx.pull()
|
||||
}
|
||||
case DropBuffer ⇒ { (ctxt, elem) ⇒
|
||||
case DropBuffer ⇒ { (ctx, elem) ⇒
|
||||
if (buffer.isFull) buffer.clear()
|
||||
buffer.enqueue(elem)
|
||||
ctxt.pull()
|
||||
ctx.pull()
|
||||
}
|
||||
case Backpressure ⇒ { (ctxt, elem) ⇒
|
||||
case Backpressure ⇒ { (ctx, elem) ⇒
|
||||
buffer.enqueue(elem)
|
||||
if (buffer.isFull) ctxt.hold()
|
||||
else ctxt.pull()
|
||||
if (buffer.isFull) ctx.hold()
|
||||
else ctx.pull()
|
||||
}
|
||||
case Error ⇒ { (ctxt, elem) ⇒
|
||||
if (buffer.isFull) ctxt.fail(new Error.BufferOverflowException(s"Buffer overflow (max capacity was: $size)!"))
|
||||
case Error ⇒ { (ctx, elem) ⇒
|
||||
if (buffer.isFull) ctx.fail(new Error.BufferOverflowException(s"Buffer overflow (max capacity was: $size)!"))
|
||||
else {
|
||||
buffer.enqueue(elem)
|
||||
ctxt.pull()
|
||||
ctx.pull()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -221,70 +221,70 @@ private[akka] final case class Buffer[T](size: Int, overflowStrategy: OverflowSt
|
|||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Completed[T]() extends DeterministicOp[T, T] {
|
||||
override def onPush(elem: T, ctxt: Context[T]): Directive = ctxt.finish()
|
||||
override def onPull(ctxt: Context[T]): Directive = ctxt.finish()
|
||||
private[akka] final case class Completed[T]() extends PushPullStage[T, T] {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = ctx.finish()
|
||||
override def onPull(ctx: Context[T]): Directive = ctx.finish()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Conflate[In, Out](seed: In ⇒ Out, aggregate: (Out, In) ⇒ Out) extends DetachedOp[In, Out] {
|
||||
private[akka] final case class Conflate[In, Out](seed: In ⇒ Out, aggregate: (Out, In) ⇒ Out) extends DetachedStage[In, Out] {
|
||||
private var agg: Any = null
|
||||
|
||||
override def onPush(elem: In, ctxt: DetachedContext[Out]): UpstreamDirective = {
|
||||
override def onPush(elem: In, ctx: DetachedContext[Out]): UpstreamDirective = {
|
||||
agg = if (agg == null) seed(elem)
|
||||
else aggregate(agg.asInstanceOf[Out], elem)
|
||||
|
||||
if (!isHolding) ctxt.pull()
|
||||
if (!ctx.isHolding) ctx.pull()
|
||||
else {
|
||||
val result = agg.asInstanceOf[Out]
|
||||
agg = null
|
||||
ctxt.pushAndPull(result)
|
||||
ctx.pushAndPull(result)
|
||||
}
|
||||
}
|
||||
|
||||
override def onPull(ctxt: DetachedContext[Out]): DownstreamDirective = {
|
||||
if (isFinishing) {
|
||||
if (agg == null) ctxt.finish()
|
||||
override def onPull(ctx: DetachedContext[Out]): DownstreamDirective = {
|
||||
if (ctx.isFinishing) {
|
||||
if (agg == null) ctx.finish()
|
||||
else {
|
||||
val result = agg.asInstanceOf[Out]
|
||||
agg = null
|
||||
ctxt.pushAndFinish(result)
|
||||
ctx.pushAndFinish(result)
|
||||
}
|
||||
} else if (agg == null) ctxt.hold()
|
||||
} else if (agg == null) ctx.hold()
|
||||
else {
|
||||
val result = agg.asInstanceOf[Out]
|
||||
agg = null
|
||||
ctxt.push(result)
|
||||
ctx.push(result)
|
||||
}
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctxt: DetachedContext[Out]): TerminationDirective = ctxt.absorbTermination()
|
||||
override def onUpstreamFinish(ctx: DetachedContext[Out]): TerminationDirective = ctx.absorbTermination()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final case class Expand[In, Out, Seed](seed: In ⇒ Seed, extrapolate: Seed ⇒ (Out, Seed)) extends DetachedOp[In, Out] {
|
||||
private[akka] final case class Expand[In, Out, Seed](seed: In ⇒ Seed, extrapolate: Seed ⇒ (Out, Seed)) extends DetachedStage[In, Out] {
|
||||
private var s: Any = null
|
||||
|
||||
override def onPush(elem: In, ctxt: DetachedContext[Out]): UpstreamDirective = {
|
||||
override def onPush(elem: In, ctx: DetachedContext[Out]): UpstreamDirective = {
|
||||
s = seed(elem)
|
||||
if (isHolding) {
|
||||
if (ctx.isHolding) {
|
||||
val (emit, newS) = extrapolate(s.asInstanceOf[Seed])
|
||||
s = newS
|
||||
ctxt.pushAndPull(emit)
|
||||
} else ctxt.hold()
|
||||
ctx.pushAndPull(emit)
|
||||
} else ctx.hold()
|
||||
}
|
||||
|
||||
override def onPull(ctxt: DetachedContext[Out]): DownstreamDirective = {
|
||||
if (s == null) ctxt.hold()
|
||||
override def onPull(ctx: DetachedContext[Out]): DownstreamDirective = {
|
||||
if (s == null) ctx.hold()
|
||||
else {
|
||||
val (emit, newS) = extrapolate(s.asInstanceOf[Seed])
|
||||
s = newS
|
||||
if (isHolding) ctxt.pushAndPull(emit)
|
||||
else ctxt.push(emit)
|
||||
if (ctx.isHolding) ctx.pushAndPull(emit)
|
||||
else ctx.push(emit)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -68,7 +68,7 @@ object FlexiMerge {
|
|||
class ReadAny(val inputs: JList[InputHandle]) extends ReadCondition
|
||||
|
||||
/**
|
||||
* Read condition for the [[MergeLogic#State]] that will be
|
||||
* Read condition for the [[FlexiMerge#State]] that will be
|
||||
* fulfilled when there are elements for any of the given upstream
|
||||
* inputs, however it differs from [[ReadAny]] in the case that both
|
||||
* the `preferred` and at least one other `secondary` input have demand,
|
||||
|
|
@ -80,7 +80,7 @@ object FlexiMerge {
|
|||
class ReadPreferred(val preferred: InputHandle, val secondaries: JList[InputHandle]) extends ReadCondition
|
||||
|
||||
/**
|
||||
* Read condition for the [[MergeLogic#State]] that will be
|
||||
* Read condition for the [[FlexiMerge#State]] that will be
|
||||
* fulfilled when there are elements for *all* of the given upstream
|
||||
* inputs.
|
||||
*
|
||||
|
|
|
|||
|
|
@ -122,13 +122,13 @@ object FlexiRoute {
|
|||
* handle cancel from downstream output.
|
||||
*
|
||||
* The `onComplete` method is called the upstream input was completed successfully.
|
||||
* It returns next behavior or [[#SameState]] to keep current behavior.
|
||||
* It returns next behavior or [[#sameState]] to keep current behavior.
|
||||
*
|
||||
* The `onError` method is called when the upstream input was completed with failure.
|
||||
* It returns next behavior or [[#SameState]] to keep current behavior.
|
||||
*
|
||||
* The `onCancel` method is called when a downstream output cancels.
|
||||
* It returns next behavior or [[#SameState]] to keep current behavior.
|
||||
* It returns next behavior or [[#sameState]] to keep current behavior.
|
||||
*/
|
||||
abstract class CompletionHandling[In] {
|
||||
def onComplete(ctx: RouteLogicContext[In, Any]): Unit
|
||||
|
|
@ -144,7 +144,7 @@ object FlexiRoute {
|
|||
* [[RouteLogicContext#emit]].
|
||||
*
|
||||
* The `onInput` method is called when an `element` was read from upstream.
|
||||
* The function returns next behavior or [[#SameState]] to keep current behavior.
|
||||
* The function returns next behavior or [[#sameState]] to keep current behavior.
|
||||
*/
|
||||
abstract class State[In, Out](val condition: DemandCondition) {
|
||||
def onInput(ctx: RouteLogicContext[In, Out], preferredOutput: OutputHandle, element: In): State[In, _]
|
||||
|
|
|
|||
|
|
@ -4,13 +4,12 @@
|
|||
package akka.stream.javadsl
|
||||
|
||||
import akka.stream._
|
||||
|
||||
import akka.japi.Util
|
||||
import akka.stream.scaladsl
|
||||
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import akka.stream.stage.Stage
|
||||
|
||||
object Flow {
|
||||
|
||||
|
|
@ -282,46 +281,31 @@ class Flow[-In, +Out](delegate: scaladsl.Flow[In, Out]) {
|
|||
new Flow(delegate.buffer(size, overflowStrategy))
|
||||
|
||||
/**
|
||||
* Generic transformation of a stream: for each element the [[akka.stream.Transformer#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
* therefore you do not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
|
||||
* This operator makes it possible to extend the `Flow` API when there is no specialized
|
||||
* operator that performs the transformation.
|
||||
*
|
||||
* Note that you can use [[#timerTransform]] if you need support for scheduled events in the transformer.
|
||||
*/
|
||||
def transform[U](name: String, mkTransformer: japi.Creator[Transformer[Out, U]]): javadsl.Flow[In, U] =
|
||||
new Flow(delegate.transform(name, () ⇒ mkTransformer.create()))
|
||||
def transform[U](name: String, mkStage: japi.Creator[Stage[Out, U]]): javadsl.Flow[In, U] =
|
||||
new Flow(delegate.transform(name, () ⇒ mkStage.create()))
|
||||
|
||||
/**
|
||||
* Transformation of a stream, with additional support for scheduled events.
|
||||
*
|
||||
* For each element the [[akka.stream.Transformer#onNext]]
|
||||
* For each element the [[akka.stream.TransformerLike#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* subscribers, the [[akka.stream.TransformerLike#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* the [[akka.stream.TransformerLike#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
* [[akka.stream.TransformerLike#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
* After normal completion or error the [[akka.stream.TransformerLike#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
|
|
@ -330,8 +314,8 @@ class Flow[-In, +Out](delegate: scaladsl.Flow[In, Out]) {
|
|||
*
|
||||
* Note that you can use [[#transform]] if you just need to transform elements time plays no role in the transformation.
|
||||
*/
|
||||
def timerTransform[U](name: String, mkTransformer: japi.Creator[TimerTransformer[Out, U]]): javadsl.Flow[In, U] =
|
||||
new Flow(delegate.timerTransform(name, () ⇒ mkTransformer.create()))
|
||||
def timerTransform[U](name: String, mkStage: japi.Creator[TimerTransformer[Out, U]]): javadsl.Flow[In, U] =
|
||||
new Flow(delegate.timerTransform(name, () ⇒ mkStage.create()))
|
||||
|
||||
/**
|
||||
* Takes up to `n` elements from the stream and returns a pair containing a strict sequence of the taken element
|
||||
|
|
|
|||
|
|
@ -4,7 +4,6 @@
|
|||
package akka.stream.javadsl
|
||||
|
||||
import java.util.concurrent.Callable
|
||||
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.japi.Util
|
||||
|
|
@ -12,13 +11,13 @@ import akka.stream._
|
|||
import akka.stream.scaladsl.PropsSource
|
||||
import org.reactivestreams.Publisher
|
||||
import org.reactivestreams.Subscriber
|
||||
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.concurrent.Future
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.language.higherKinds
|
||||
import scala.language.implicitConversions
|
||||
import akka.stream.stage.Stage
|
||||
|
||||
/** Java API */
|
||||
object Source {
|
||||
|
|
@ -387,56 +386,41 @@ class Source[+Out](delegate: scaladsl.Source[Out]) {
|
|||
new Source(delegate.buffer(size, overflowStrategy))
|
||||
|
||||
/**
|
||||
* Generic transformation of a stream: for each element the [[akka.stream.Transformer#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
* therefore you do not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
|
||||
* This operator makes it possible to extend the `Flow` API when there is no specialized
|
||||
* operator that performs the transformation.
|
||||
*
|
||||
* Note that you can use [[#timerTransform]] if you need support for scheduled events in the transformer.
|
||||
*/
|
||||
def transform[U](name: String, mkTransformer: japi.Creator[Transformer[Out, U]]): javadsl.Source[U] =
|
||||
new Source(delegate.transform(name, () ⇒ mkTransformer.create()))
|
||||
def transform[U](name: String, mkStage: japi.Creator[Stage[Out, U]]): javadsl.Source[U] =
|
||||
new Source(delegate.transform(name, () ⇒ mkStage.create()))
|
||||
|
||||
/**
|
||||
* Transformation of a stream, with additional support for scheduled events.
|
||||
*
|
||||
* For each element the [[akka.stream.Transformer#onNext]]
|
||||
* For each element the [[akka.stream.TransformerLike#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* subscribers, the [[akka.stream.TransformerLike#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* the [[akka.stream.TransformerLike#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
* [[akka.stream.TransformerLike#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
* After normal completion or error the [[akka.stream.TransformerLike#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
* It is possible to keep state in the concrete [[akka.stream.TimerTransformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.TimerTransformer]] is executed by an actor and
|
||||
* therefore you do not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
*
|
||||
* Note that you can use [[#transform]] if you just need to transform elements time plays no role in the transformation.
|
||||
*/
|
||||
def timerTransform[U](name: String, mkTransformer: japi.Creator[TimerTransformer[Out, U]]): javadsl.Source[U] =
|
||||
new Source(delegate.timerTransform(name, () ⇒ mkTransformer.create()))
|
||||
def timerTransform[U](name: String, mkStage: japi.Creator[TimerTransformer[Out, U]]): javadsl.Source[U] =
|
||||
new Source(delegate.timerTransform(name, () ⇒ mkStage.create()))
|
||||
|
||||
/**
|
||||
* Takes up to `n` elements from the stream and returns a pair containing a strict sequence of the taken element
|
||||
|
|
|
|||
|
|
@ -5,15 +5,14 @@ package akka.stream.scaladsl
|
|||
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.annotation.unchecked.uncheckedVariance
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import org.reactivestreams.{ Publisher, Subscriber, Subscription }
|
||||
import akka.stream.Transformer
|
||||
import akka.stream.impl.{ ActorBasedFlowMaterializer, ActorProcessorFactory, FanoutProcessorImpl, BlackholeSubscriber }
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.stream.stage._
|
||||
|
||||
sealed trait ActorFlowSink[-In] extends Sink[In] {
|
||||
|
||||
|
|
@ -167,15 +166,15 @@ object OnCompleteSink {
|
|||
final case class OnCompleteSink[In](callback: Try[Unit] ⇒ Unit) extends SimpleActorFlowSink[In] {
|
||||
|
||||
override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) =
|
||||
Source(flowPublisher).transform("onCompleteSink", () ⇒ new Transformer[In, Unit] {
|
||||
override def onNext(in: In) = Nil
|
||||
override def onError(e: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = {
|
||||
e match {
|
||||
case None ⇒ callback(OnCompleteSink.SuccessUnit)
|
||||
case Some(e) ⇒ callback(Failure(e))
|
||||
Source(flowPublisher).transform("onCompleteSink", () ⇒ new PushStage[In, Unit] {
|
||||
override def onPush(elem: In, ctx: Context[Unit]): Directive = ctx.pull()
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[Unit]): TerminationDirective = {
|
||||
callback(Failure(cause))
|
||||
ctx.fail(cause)
|
||||
}
|
||||
Nil
|
||||
override def onUpstreamFinish(ctx: Context[Unit]): TerminationDirective = {
|
||||
callback(OnCompleteSink.SuccessUnit)
|
||||
ctx.finish()
|
||||
}
|
||||
}).to(BlackholeSink).run()(materializer.withNamePrefix(flowName))
|
||||
}
|
||||
|
|
@ -191,15 +190,18 @@ final case class ForeachSink[In](f: In ⇒ Unit) extends KeyedActorFlowSink[In]
|
|||
|
||||
override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = {
|
||||
val promise = Promise[Unit]()
|
||||
Source(flowPublisher).transform("foreach", () ⇒ new Transformer[In, Unit] {
|
||||
override def onNext(in: In) = { f(in); Nil }
|
||||
override def onError(cause: Throwable): Unit = ()
|
||||
override def onTermination(e: Option[Throwable]) = {
|
||||
e match {
|
||||
case None ⇒ promise.success(())
|
||||
case Some(e) ⇒ promise.failure(e)
|
||||
Source(flowPublisher).transform("foreach", () ⇒ new PushStage[In, Unit] {
|
||||
override def onPush(elem: In, ctx: Context[Unit]): Directive = {
|
||||
f(elem)
|
||||
ctx.pull()
|
||||
}
|
||||
Nil
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[Unit]): TerminationDirective = {
|
||||
promise.failure(cause)
|
||||
ctx.fail(cause)
|
||||
}
|
||||
override def onUpstreamFinish(ctx: Context[Unit]): TerminationDirective = {
|
||||
promise.success(())
|
||||
ctx.finish()
|
||||
}
|
||||
}).to(BlackholeSink).run()(materializer.withNamePrefix(flowName))
|
||||
promise.future
|
||||
|
|
@ -220,16 +222,22 @@ final case class FoldSink[U, In](zero: U)(f: (U, In) ⇒ U) extends KeyedActorFl
|
|||
override def attach(flowPublisher: Publisher[In], materializer: ActorBasedFlowMaterializer, flowName: String) = {
|
||||
val promise = Promise[U]()
|
||||
|
||||
Source(flowPublisher).transform("fold", () ⇒ new Transformer[In, U] {
|
||||
var state: U = zero
|
||||
override def onNext(in: In): immutable.Seq[U] = { state = f(state, in); Nil }
|
||||
override def onError(cause: Throwable) = ()
|
||||
override def onTermination(e: Option[Throwable]) = {
|
||||
e match {
|
||||
case None ⇒ promise.success(state)
|
||||
case Some(e) ⇒ promise.failure(e)
|
||||
Source(flowPublisher).transform("fold", () ⇒ new PushStage[In, U] {
|
||||
private var aggregator = zero
|
||||
|
||||
override def onPush(elem: In, ctx: Context[U]): Directive = {
|
||||
aggregator = f(aggregator, elem)
|
||||
ctx.pull()
|
||||
}
|
||||
Nil
|
||||
|
||||
override def onUpstreamFailure(cause: Throwable, ctx: Context[U]): TerminationDirective = {
|
||||
promise.failure(cause)
|
||||
ctx.fail(cause)
|
||||
}
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[U]): TerminationDirective = {
|
||||
promise.success(aggregator)
|
||||
ctx.finish()
|
||||
}
|
||||
}).to(BlackholeSink).run()(materializer.withNamePrefix(flowName))
|
||||
|
||||
|
|
|
|||
|
|
@ -107,7 +107,7 @@ object FlexiMerge {
|
|||
|
||||
/**
|
||||
* The possibly stateful logic that reads from input via the defined [[MergeLogic#State]] and
|
||||
* handles completion and error via the defined [[FlexiMerge#CompletionHandling]].
|
||||
* handles completion and error via the defined [[MergeLogic#CompletionHandling]].
|
||||
*
|
||||
* Concrete instance is supposed to be created by implementing [[FlexiMerge#createMergeLogic]].
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
package akka.stream.scaladsl
|
||||
|
||||
import akka.stream.impl.Ast._
|
||||
import akka.stream.{ TimerTransformer, Transformer, OverflowStrategy }
|
||||
import akka.stream.{ TimerTransformer, TransformerLike, OverflowStrategy }
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.{ Duration, FiniteDuration }
|
||||
|
|
@ -12,6 +12,7 @@ import scala.concurrent.Future
|
|||
import scala.language.higherKinds
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.FlattenStrategy
|
||||
import akka.stream.stage._
|
||||
|
||||
/**
|
||||
* A `Flow` is a set of stream processing steps that has one open input and one open output.
|
||||
|
|
@ -217,8 +218,8 @@ trait FlowOps[+Out] {
|
|||
timerTransform("dropWithin", () ⇒ new TimerTransformer[Out, Out] {
|
||||
scheduleOnce(DropWithinTimerKey, d)
|
||||
|
||||
var delegate: Transformer[Out, Out] =
|
||||
new Transformer[Out, Out] {
|
||||
var delegate: TransformerLike[Out, Out] =
|
||||
new TransformerLike[Out, Out] {
|
||||
def onNext(in: Out) = Nil
|
||||
}
|
||||
|
||||
|
|
@ -253,7 +254,7 @@ trait FlowOps[+Out] {
|
|||
timerTransform("takeWithin", () ⇒ new TimerTransformer[Out, Out] {
|
||||
scheduleOnce(TakeWithinTimerKey, d)
|
||||
|
||||
var delegate: Transformer[Out, Out] = FlowOps.identityTransformer[Out]
|
||||
var delegate: TransformerLike[Out, Out] = FlowOps.identityTransformer[Out]
|
||||
|
||||
override def onNext(in: Out) = delegate.onNext(in)
|
||||
override def isComplete = delegate.isComplete
|
||||
|
|
@ -305,30 +306,14 @@ trait FlowOps[+Out] {
|
|||
andThen(Buffer(size, overflowStrategy))
|
||||
|
||||
/**
|
||||
* Generic transformation of a stream: for each element the [[akka.stream.Transformer#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
* therefore you do not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
* Generic transformation of a stream with a custom processing [[akka.stream.stage.Stage]].
|
||||
* This operator makes it possible to extend the `Flow` API when there is no specialized
|
||||
* operator that performs the transformation.
|
||||
*
|
||||
* Note that you can use [[#timerTransform]] if you need support for scheduled events in the transformer.
|
||||
*/
|
||||
def transform[T](name: String, mkTransformer: () ⇒ Transformer[Out, T]): Repr[T] = {
|
||||
andThen(Transform(name, mkTransformer.asInstanceOf[() ⇒ Transformer[Any, Any]]))
|
||||
}
|
||||
def transform[T](name: String, mkStage: () ⇒ Stage[Out, T]): Repr[T] =
|
||||
andThen(StageFactory(mkStage, name))
|
||||
|
||||
/**
|
||||
* Takes up to `n` elements from the stream and returns a pair containing a strict sequence of the taken element
|
||||
|
|
@ -381,19 +366,19 @@ trait FlowOps[+Out] {
|
|||
/**
|
||||
* Transformation of a stream, with additional support for scheduled events.
|
||||
*
|
||||
* For each element the [[akka.stream.Transformer#onNext]]
|
||||
* For each element the [[akka.stream.TransformerLike#onNext]]
|
||||
* function is invoked, expecting a (possibly empty) sequence of output elements
|
||||
* to be produced.
|
||||
* After handing off the elements produced from one input element to the downstream
|
||||
* subscribers, the [[akka.stream.Transformer#isComplete]] predicate determines whether to end
|
||||
* subscribers, the [[akka.stream.TransformerLike#isComplete]] predicate determines whether to end
|
||||
* stream processing at this point; in that case the upstream subscription is
|
||||
* canceled. Before signaling normal completion to the downstream subscribers,
|
||||
* the [[akka.stream.Transformer#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* the [[akka.stream.TransformerLike#onTermination]] function is invoked to produce a (possibly empty)
|
||||
* sequence of elements in response to the end-of-stream event.
|
||||
*
|
||||
* [[akka.stream.Transformer#onError]] is called when failure is signaled from upstream.
|
||||
* [[akka.stream.TransformerLike#onError]] is called when failure is signaled from upstream.
|
||||
*
|
||||
* After normal completion or error the [[akka.stream.Transformer#cleanup]] function is called.
|
||||
* After normal completion or error the [[akka.stream.TransformerLike#cleanup]] function is called.
|
||||
*
|
||||
* It is possible to keep state in the concrete [[akka.stream.Transformer]] instance with
|
||||
* ordinary instance variables. The [[akka.stream.Transformer]] is executed by an actor and
|
||||
|
|
@ -402,8 +387,8 @@ trait FlowOps[+Out] {
|
|||
*
|
||||
* Note that you can use [[#transform]] if you just need to transform elements time plays no role in the transformation.
|
||||
*/
|
||||
def timerTransform[U](name: String, mkTransformer: () ⇒ TimerTransformer[Out, U]): Repr[U] =
|
||||
andThen(TimerTransform(name, mkTransformer.asInstanceOf[() ⇒ TimerTransformer[Any, Any]]))
|
||||
def timerTransform[U](name: String, mkStage: () ⇒ TimerTransformer[Out, U]): Repr[U] =
|
||||
andThen(TimerTransform(mkStage.asInstanceOf[() ⇒ TimerTransformer[Any, Any]], name))
|
||||
|
||||
/** INTERNAL API */
|
||||
// Storing ops in reverse order
|
||||
|
|
@ -418,16 +403,20 @@ private[stream] object FlowOps {
|
|||
private case object DropWithinTimerKey
|
||||
private case object GroupedWithinTimerKey
|
||||
|
||||
private[this] final case object CompletedTransformer extends Transformer[Any, Any] {
|
||||
private[this] final case object CompletedTransformer extends TransformerLike[Any, Any] {
|
||||
override def onNext(elem: Any) = Nil
|
||||
override def isComplete = true
|
||||
}
|
||||
|
||||
private[this] final case object IdentityTransformer extends Transformer[Any, Any] {
|
||||
private[this] final case object IdentityTransformer extends TransformerLike[Any, Any] {
|
||||
override def onNext(elem: Any) = List(elem)
|
||||
}
|
||||
|
||||
def completedTransformer[T]: Transformer[T, T] = CompletedTransformer.asInstanceOf[Transformer[T, T]]
|
||||
def identityTransformer[T]: Transformer[T, T] = IdentityTransformer.asInstanceOf[Transformer[T, T]]
|
||||
def completedTransformer[T]: TransformerLike[T, T] = CompletedTransformer.asInstanceOf[TransformerLike[T, T]]
|
||||
def identityTransformer[T]: TransformerLike[T, T] = IdentityTransformer.asInstanceOf[TransformerLike[T, T]]
|
||||
|
||||
def identityStage[T]: Stage[T, T] = new PushStage[T, T] {
|
||||
override def onPush(elem: T, ctx: Context[T]): Directive = ctx.push(elem)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import scala.annotation.unchecked.uncheckedVariance
|
|||
import scala.language.{ existentials, higherKinds }
|
||||
import akka.stream.FlowMaterializer
|
||||
|
||||
private[stream] object Pipe {
|
||||
private[akka] object Pipe {
|
||||
private val emptyInstance = Pipe[Any, Any](ops = Nil)
|
||||
def empty[T]: Pipe[T, T] = emptyInstance.asInstanceOf[Pipe[T, T]]
|
||||
}
|
||||
|
|
@ -16,7 +16,7 @@ private[stream] object Pipe {
|
|||
/**
|
||||
* Flow with one open input and one open output.
|
||||
*/
|
||||
private[stream] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flow[In, Out] {
|
||||
private[akka] final case class Pipe[-In, +Out](ops: List[AstNode]) extends Flow[In, Out] {
|
||||
override type Repr[+O] = Pipe[In @uncheckedVariance, O]
|
||||
|
||||
override private[scaladsl] def andThen[U](op: AstNode): Repr[U] = this.copy(ops = op :: ops) // FIXME raw addition of AstNodes
|
||||
|
|
|
|||
434
akka-stream/src/main/scala/akka/stream/stage/Stage.scala
Normal file
434
akka-stream/src/main/scala/akka/stream/stage/Stage.scala
Normal file
|
|
@ -0,0 +1,434 @@
|
|||
/**
|
||||
* Copyright (C) 2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.stream.stage
|
||||
|
||||
/**
|
||||
* General interface for stream transformation.
|
||||
*
|
||||
* Custom `Stage` implementations are intended to be used with
|
||||
* [[akka.stream.scaladsl.FlowOps#transform]] or
|
||||
* [[akka.stream.javadsl.Flow#transform]] to extend the `Flow` API when there
|
||||
* is no specialized operator that performs the transformation.
|
||||
*
|
||||
* Custom implementations are subclasses of [[PushPullStage]] or
|
||||
* [[DetachedStage]]. Sometimes it is convenient to extend
|
||||
* [[StatefulStage]] for support of become like behavior.
|
||||
*
|
||||
* It is possible to keep state in the concrete `Stage` instance with
|
||||
* ordinary instance variables. The `Transformer` is executed by an actor and
|
||||
* therefore you don not have to add any additional thread safety or memory
|
||||
* visibility constructs to access the state from the callback methods.
|
||||
*
|
||||
* @see [[akka.stream.scaladsl.Flow#transform]]
|
||||
* @see [[akka.stream.javadsl.Flow#transform]]
|
||||
*/
|
||||
sealed trait Stage[-In, Out]
|
||||
|
||||
private[stream] abstract class AbstractStage[-In, Out, PushD <: Directive, PullD <: Directive, Ctx <: Context[Out]] extends Stage[In, Out] {
|
||||
private[stream] var holding = false
|
||||
private[stream] var allowedToPush = false
|
||||
private[stream] var terminationPending = false
|
||||
|
||||
/**
|
||||
* `onPush` is called when an element from upstream is available and there is demand from downstream, i.e.
|
||||
* in `onPush` you are allowed to call [[akka.stream.stage.Context#push]] to emit one element downstreams,
|
||||
* or you can absorb the element by calling [[akka.stream.stage.Context#pull]]. Note that you can only
|
||||
* emit zero or one element downstream from `onPull`.
|
||||
*
|
||||
* To emit more than one element you have to push the remaining elements from [[#onPull]], one-by-one.
|
||||
* `onPush` is not called again until `onPull` has requested more elements with
|
||||
* [[akka.stream.stage.Context#pull]].
|
||||
*/
|
||||
def onPush(elem: In, ctx: Ctx): PushD
|
||||
|
||||
/**
|
||||
* `onPull` is called when there is demand from downstream, i.e. you are allowed to push one element
|
||||
* downstreams with [[akka.stream.stage.Context#push]], or request elements from upstreams with
|
||||
* [[akka.stream.stage.Context#pull]]
|
||||
*/
|
||||
def onPull(ctx: Ctx): PullD
|
||||
|
||||
/**
|
||||
* `onUpstreamFinish` is called when upstream has signaled that the stream is
|
||||
* successfully completed. Here you cannot call [[akka.stream.stage.Context#push]],
|
||||
* because there might not be any demand from downstream. To emit additional elements before
|
||||
* terminating you can use [[akka.stream.stage.Context#absorbTermination]] and push final elements
|
||||
* from [[#onPull]]. The stage will then be in finishing state, which can be checked
|
||||
* with [[akka.stream.stage.Context#isFinishing]].
|
||||
*
|
||||
* By default the finish signal is immediately propagated with [[akka.stream.stage.Context#finish]].
|
||||
*/
|
||||
def onUpstreamFinish(ctx: Ctx): TerminationDirective = ctx.finish()
|
||||
|
||||
/**
|
||||
* `onDownstreamFinish` is called when downstream has cancelled.
|
||||
*
|
||||
* By default the cancel signal is immediately propagated with [[akka.stream.stage.Context#finish]].
|
||||
*/
|
||||
def onDownstreamFinish(ctx: Ctx): TerminationDirective = ctx.finish()
|
||||
|
||||
/**
|
||||
* `onUpstreamFailure` is called when upstream has signaled that the stream is completed
|
||||
* with error. It is not called if [[#onPull]] or [[#onPush]] of the stage itself
|
||||
* throws an exception.
|
||||
*
|
||||
* Note that elements that were emitted by upstream before the error happened might
|
||||
* not have been received by this stage when `onUpstreamFailure` is called, i.e.
|
||||
* errors are not backpressured and might be propagated as soon as possible.
|
||||
*
|
||||
* Here you cannot call [[akka.stream.stage.Context#push]], because there might not
|
||||
* be any demand from downstream. To emit additional elements before terminating you
|
||||
* can use [[akka.stream.stage.Context#absorbTermination]] and push final elements
|
||||
* from [[#onPull]]. The stage will then be in finishing state, which can be checked
|
||||
* with [[akka.stream.stage.Context#isFinishing]].
|
||||
*/
|
||||
def onUpstreamFailure(cause: Throwable, ctx: Ctx): TerminationDirective = ctx.fail(cause)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* `PushPullStage` implementations participate in 1-bounded regions. For every external non-completion signal these
|
||||
* stages produce *exactly one* push or pull signal.
|
||||
*
|
||||
* [[#onPush]] is called when an element from upstream is available and there is demand from downstream, i.e.
|
||||
* in `onPush` you are allowed to call [[Context#push]] to emit one element downstreams, or you can absorb the
|
||||
* element by calling [[Context#pull]]. Note that you can only emit zero or one element downstream from `onPull`.
|
||||
* To emit more than one element you have to push the remaining elements from [[#onPull]], one-by-one.
|
||||
* `onPush` is not called again until `onPull` has requested more elements with [[Context#pull]].
|
||||
*
|
||||
* [[StatefulStage]] has support for making it easy to emit more than one element from `onPush`.
|
||||
*
|
||||
* [[#onPull]] is called when there is demand from downstream, i.e. you are allowed to push one element
|
||||
* downstreams with [[Context#push]], or request elements from upstreams with [[Context#pull]]. If you
|
||||
* always perform transitive pull by calling `ctx.pull` from `onPull` you can use [[PushStage]] instead of
|
||||
* `PushPullStage`.
|
||||
*
|
||||
* Stages are allowed to do early completion of downstream and cancel of upstream. This is done with [[Context#finish]],
|
||||
* which is a combination of cancel/complete.
|
||||
*
|
||||
* Since onComplete is not a backpressured signal it is sometimes preferable to push a final element and then
|
||||
* immediately finish. This combination is exposed as [[Context#pushAndFinish]] which enables stages to
|
||||
* propagate completion events without waiting for an extra round of pull.
|
||||
*
|
||||
* Another peculiarity is how to convert termination events (complete/failure) into elements. The problem
|
||||
* here is that the termination events are not backpressured while elements are. This means that simply calling
|
||||
* [[Context#push]] as a response to [[#onUpstreamFinish]] or [[#onUpstreamFailure]] will very likely break boundedness
|
||||
* and result in a buffer overflow somewhere. Therefore the only allowed command in this case is
|
||||
* [[Context#absorbTermination]] which stops the propagation of the termination signal, and puts the stage in a
|
||||
* [[akka.stream.stage.Context#isFinishing]] state. Depending on whether the stage has a pending pull signal it
|
||||
* has not yet "consumed" by a push its [[#onPull]] handler might be called immediately or later. From
|
||||
* [[#onPull]] final elements can be pushed before completing downstream with [[Context#finish]] or
|
||||
* [[Context#pushAndFinish]].
|
||||
*
|
||||
* [[StatefulStage]] has support for making it easy to emit final elements.
|
||||
*
|
||||
* All these rules are enforced by types and runtime checks where needed. Always return the `Directive`
|
||||
* from the call to the [[Context]] method, and do only call [[Context]] commands once per callback.
|
||||
*
|
||||
* @see [[DetachedStage]]
|
||||
* @see [[StatefulStage]]
|
||||
* @see [[PushStage]]
|
||||
*/
|
||||
abstract class PushPullStage[In, Out] extends AbstractStage[In, Out, Directive, Directive, Context[Out]]
|
||||
|
||||
/**
|
||||
* `PushStage` is a [[PushPullStage]] that always perform transitive pull by calling `ctx.pull` from `onPull`.
|
||||
*/
|
||||
abstract class PushStage[In, Out] extends PushPullStage[In, Out] {
|
||||
/**
|
||||
* Always pulls from upstream.
|
||||
*/
|
||||
final override def onPull(ctx: Context[Out]): Directive = ctx.pull()
|
||||
}
|
||||
|
||||
/**
|
||||
* `DetachedStage` can be used to implement operations similar to [[akka.stream.scaladsl.FlowOps#buffer buffer]],
|
||||
* [[akka.stream.scaladsl.FlowOps#expand expand]] and [[akka.stream.scaladsl.FlowOps#conflate conflate]].
|
||||
*
|
||||
* `DetachedStage` implementations are boundaries between 1-bounded regions. This means that they need to enforce the
|
||||
* "exactly one" property both on their upstream and downstream regions. As a consequence a `DetachedStage` can never
|
||||
* answer an [[#onPull]] with a [[Context#pull]] or answer an [[#onPush]] with a [[Context#push]] since such an action
|
||||
* would "steal" the event from one region (resulting in zero signals) and would inject it to the other region
|
||||
* (resulting in two signals).
|
||||
*
|
||||
* However, DetachedStages have the ability to call [[akka.stream.stage.DetachedContext#hold]] as a response to
|
||||
* [[#onPush]] and [[akka.stream.stage.DetachedContext##onPull]] which temporarily takes the signal off and
|
||||
* stops execution, at the same time putting the stage in an [[akka.stream.stage.DetachedContext#isHolding]] state.
|
||||
* If the stage is in a holding state it contains one absorbed signal, therefore in this state the only possible
|
||||
* command to call is [[akka.stream.stage.DetachedContext#pushAndPull]] which results in two events making the
|
||||
* balance right again: 1 hold + 1 external event = 2 external event
|
||||
*
|
||||
* This mechanism allows synchronization between the upstream and downstream regions which otherwise can progress
|
||||
* independently.
|
||||
*
|
||||
* @see [[PushPullStage]]
|
||||
*/
|
||||
abstract class DetachedStage[In, Out] extends AbstractStage[In, Out, UpstreamDirective, DownstreamDirective, DetachedContext[Out]]
|
||||
|
||||
/**
|
||||
* The behavior of [[StatefulStage]] is defined by these two methods, which
|
||||
* has the same sematics as corresponding methods in [[PushPullStage]].
|
||||
*/
|
||||
abstract class StageState[In, Out] {
|
||||
def onPush(elem: In, ctx: Context[Out]): Directive
|
||||
def onPull(ctx: Context[Out]): Directive = ctx.pull()
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object StatefulStage {
|
||||
sealed trait AndThen
|
||||
case object Finish extends AndThen
|
||||
final case class Become(state: StageState[Any, Any]) extends AndThen
|
||||
case object Stay extends AndThen
|
||||
}
|
||||
|
||||
/**
|
||||
* `StatefulStage` is a [[PushPullStage]] that provides convenience to make some things easier.
|
||||
*
|
||||
* The behavior is defined in [[StageState]] instances. The initial behavior is specified
|
||||
* by subclass implementing the [[#initial]] method. The behavior can be changed by using [[#become]].
|
||||
*
|
||||
* Use [[#emit]] or [[#emitAndFinish]] to push more than one element from [[StageState#onPush]] or
|
||||
* [[StageState#onPull]].
|
||||
*
|
||||
* Use [[#terminationEmit]] to push final elements from [[#onUpstreamFinish]] or [[#onUpstreamFailure]].
|
||||
*/
|
||||
abstract class StatefulStage[In, Out] extends PushPullStage[In, Out] {
|
||||
import StatefulStage._
|
||||
|
||||
/**
|
||||
* Scala API
|
||||
*/
|
||||
abstract class State extends StageState[In, Out]
|
||||
|
||||
private var emitting = false
|
||||
private var _current: StageState[In, Out] = _
|
||||
become(initial)
|
||||
|
||||
/**
|
||||
* Concrete subclass must return the initial behavior from this method.
|
||||
*/
|
||||
def initial: StageState[In, Out]
|
||||
|
||||
/**
|
||||
* Current state.
|
||||
*/
|
||||
final def current: StageState[In, Out] = _current
|
||||
|
||||
/**
|
||||
* Change the behavior to another [[StageState]].
|
||||
*/
|
||||
final def become(state: StageState[In, Out]): Unit = {
|
||||
require(state ne null, "New state must not be null")
|
||||
_current = state
|
||||
}
|
||||
|
||||
/**
|
||||
* Invokes current state.
|
||||
*/
|
||||
final override def onPush(elem: In, ctx: Context[Out]): Directive = _current.onPush(elem, ctx)
|
||||
/**
|
||||
* Invokes current state.
|
||||
*/
|
||||
final override def onPull(ctx: Context[Out]): Directive = _current.onPull(ctx)
|
||||
|
||||
override def onUpstreamFinish(ctx: Context[Out]): TerminationDirective =
|
||||
if (emitting) ctx.absorbTermination()
|
||||
else ctx.finish()
|
||||
|
||||
/**
|
||||
* Scala API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams.
|
||||
*/
|
||||
final def emit(iter: Iterator[Out], ctx: Context[Out]): Directive = emit(iter, ctx, _current)
|
||||
|
||||
/**
|
||||
* Java API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams.
|
||||
*/
|
||||
final def emit(iter: java.util.Iterator[Out], ctx: Context[Out]): Directive = {
|
||||
import scala.collection.JavaConverters._
|
||||
emit(iter.asScala, ctx)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams and after that change behavior.
|
||||
*/
|
||||
final def emit(iter: Iterator[Out], ctx: Context[Out], nextState: StageState[In, Out]): Directive = {
|
||||
if (emitting) throw new IllegalStateException("already in emitting state")
|
||||
if (iter.isEmpty) {
|
||||
become(nextState)
|
||||
ctx.pull()
|
||||
} else {
|
||||
val elem = iter.next()
|
||||
if (iter.hasNext) {
|
||||
emitting = true
|
||||
become(emittingState(iter, andThen = Become(nextState.asInstanceOf[StageState[Any, Any]])))
|
||||
}
|
||||
ctx.push(elem)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams and after that change behavior.
|
||||
*/
|
||||
final def emit(iter: java.util.Iterator[Out], ctx: Context[Out], nextState: StageState[In, Out]): Directive = {
|
||||
import scala.collection.JavaConverters._
|
||||
emit(iter.asScala, ctx, nextState)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams and after that finish (complete downstreams, cancel upstreams).
|
||||
*/
|
||||
final def emitAndFinish(iter: Iterator[Out], ctx: Context[Out]): Directive = {
|
||||
if (emitting) throw new IllegalStateException("already in emitting state")
|
||||
if (iter.isEmpty)
|
||||
ctx.finish()
|
||||
else {
|
||||
val elem = iter.next()
|
||||
if (iter.hasNext) {
|
||||
emitting = true
|
||||
become(emittingState(iter, andThen = Finish))
|
||||
ctx.push(elem)
|
||||
} else
|
||||
ctx.pushAndFinish(elem)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Can be used from [[StageState#onPush]] or [[StageState#onPull]] to push more than one
|
||||
* element downstreams and after that finish (complete downstreams, cancel upstreams).
|
||||
*/
|
||||
final def emitAndFinish(iter: java.util.Iterator[Out], ctx: Context[Out]): Directive = {
|
||||
import scala.collection.JavaConverters._
|
||||
emitAndFinish(iter.asScala, ctx)
|
||||
}
|
||||
|
||||
/**
|
||||
* Scala API: Can be used from [[#onUpstreamFinish]] to push final elements downstreams
|
||||
* before completing the stream successfully. Note that if this is used from
|
||||
* [[#onUpstreamFailure]] the error will be absorbed and the stream will be completed
|
||||
* successfully.
|
||||
*/
|
||||
final def terminationEmit(iter: Iterator[Out], ctx: Context[Out]): TerminationDirective = {
|
||||
val empty = iter.isEmpty
|
||||
if (empty && emitting) ctx.absorbTermination()
|
||||
else if (empty) ctx.finish()
|
||||
else {
|
||||
become(emittingState(iter, andThen = Finish))
|
||||
ctx.absorbTermination()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Can be used from [[#onUpstreamFinish]] or [[#onUpstreamFailure]] to push final
|
||||
* elements downstreams.
|
||||
*/
|
||||
final def terminationEmit(iter: java.util.Iterator[Out], ctx: Context[Out]): TerminationDirective = {
|
||||
import scala.collection.JavaConverters._
|
||||
terminationEmit(iter.asScala, ctx)
|
||||
}
|
||||
|
||||
private def emittingState(iter: Iterator[Out], andThen: AndThen) = new State {
|
||||
override def onPush(elem: In, ctx: Context[Out]) = throw new IllegalStateException("onPush not allowed in emittingState")
|
||||
override def onPull(ctx: Context[Out]) = {
|
||||
if (iter.hasNext) {
|
||||
val elem = iter.next()
|
||||
if (iter.hasNext)
|
||||
ctx.push(elem)
|
||||
else if (!ctx.isFinishing) {
|
||||
emitting = false
|
||||
andThen match {
|
||||
case Stay ⇒ // ok
|
||||
case Become(newState) ⇒ become(newState.asInstanceOf[StageState[In, Out]])
|
||||
case Finish ⇒ ctx.pushAndFinish(elem)
|
||||
}
|
||||
ctx.push(elem)
|
||||
} else
|
||||
ctx.pushAndFinish(elem)
|
||||
} else
|
||||
throw new IllegalStateException("onPull with empty iterator is not expected in emittingState")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Return type from [[Context]] methods.
|
||||
*/
|
||||
sealed trait Directive
|
||||
sealed trait UpstreamDirective extends Directive
|
||||
sealed trait DownstreamDirective extends Directive
|
||||
sealed trait TerminationDirective extends Directive
|
||||
final class FreeDirective extends UpstreamDirective with DownstreamDirective with TerminationDirective
|
||||
|
||||
/**
|
||||
* Passed to the callback methods of [[PushPullStage]] and [[StatefulStage]].
|
||||
*/
|
||||
sealed trait Context[Out] {
|
||||
/**
|
||||
* Push one element to downstreams.
|
||||
*/
|
||||
def push(elem: Out): DownstreamDirective
|
||||
/**
|
||||
* Request for more elements from upstreams.
|
||||
*/
|
||||
def pull(): UpstreamDirective
|
||||
/**
|
||||
* Cancel upstreams and complete downstreams successfully.
|
||||
*/
|
||||
def finish(): FreeDirective
|
||||
/**
|
||||
* Push one element to downstream immediately followed by
|
||||
* cancel of upstreams and complete of downstreams.
|
||||
*/
|
||||
def pushAndFinish(elem: Out): DownstreamDirective
|
||||
/**
|
||||
* Cancel upstreams and complete downstreams with failure.
|
||||
*/
|
||||
def fail(cause: Throwable): FreeDirective
|
||||
/**
|
||||
* Puts the stage in a finishing state so that
|
||||
* final elements can be pushed from `onPull`.
|
||||
*/
|
||||
def absorbTermination(): TerminationDirective
|
||||
|
||||
/**
|
||||
* This returns `true` after [[#absorbTermination]] has been used.
|
||||
*/
|
||||
def isFinishing: Boolean
|
||||
}
|
||||
|
||||
/**
|
||||
* Passed to the callback methods of [[DetachedStage]].
|
||||
*
|
||||
* [[#hold]] stops execution and at the same time putting the stage in a holding state.
|
||||
* If the stage is in a holding state it contains one absorbed signal, therefore in
|
||||
* this state the only possible command to call is [[#pushAndPull]] which results in two
|
||||
* events making the balance right again: 1 hold + 1 external event = 2 external event
|
||||
*/
|
||||
trait DetachedContext[Out] extends Context[Out] {
|
||||
def hold(): FreeDirective
|
||||
|
||||
/**
|
||||
* This returns `true` when [[#hold]] has been used
|
||||
* and it is reset to `false` after [[#pushAndPull]].
|
||||
*/
|
||||
def isHolding: Boolean
|
||||
|
||||
def pushAndPull(elem: Out): FreeDirective
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] trait BoundaryContext extends Context[Any] {
|
||||
def exit(): FreeDirective
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue