Merge branch 'wip-RIP-OneBoundedInterpreter2' into release-2.3-dev
This commit is contained in:
commit
06ce968b16
107 changed files with 2456 additions and 3061 deletions
|
|
@ -81,7 +81,7 @@ public interface HttpEntity {
|
|||
/**
|
||||
* Returns a stream of data bytes this entity consists of.
|
||||
*/
|
||||
public abstract Source<ByteString, ?> getDataBytes();
|
||||
public abstract Source<ByteString, Object> getDataBytes();
|
||||
|
||||
/**
|
||||
* Returns a future of a strict entity that contains the same data as this entity
|
||||
|
|
|
|||
|
|
@ -117,7 +117,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
|
||||
val shape = new FanInShape2(requests, responses, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
passAlong(requests, out, doFinish = false, doFail = true)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
|
||||
|
|
@ -147,7 +147,7 @@ private[http] object OutgoingConnectionBlueprint {
|
|||
|
||||
val shape = new FanInShape2(dataInput, methodBypassInput, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
// each connection uses a single (private) response parser instance for all its responses
|
||||
// which builds a cache of all header instances seen on that connection
|
||||
val parser = rootParser.createShallowCopy()
|
||||
|
|
|
|||
|
|
@ -115,7 +115,7 @@ private object PoolConductor {
|
|||
|
||||
override val shape = new FanInShape2(ctxIn, slotIn, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
val slotStates = Array.fill[SlotState](slotCount)(Unconnected)
|
||||
var nextSlot = 0
|
||||
|
||||
|
|
@ -207,7 +207,7 @@ private object PoolConductor {
|
|||
|
||||
override val shape = new UniformFanOutShape[SwitchCommand, RequestContext](slotCount)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
shape.outArray foreach { setHandler(_, ignoreTerminateOutput) }
|
||||
|
||||
val in = shape.in
|
||||
|
|
|
|||
|
|
@ -57,10 +57,10 @@ private object PoolSlot {
|
|||
import FlowGraph.Implicits._
|
||||
|
||||
val slotProcessor = b.add {
|
||||
Flow[RequestContext].andThenMat { () ⇒
|
||||
Flow.fromProcessor { () ⇒
|
||||
val actor = system.actorOf(Props(new SlotProcessor(slotIx, connectionFlow, settings)).withDeploy(Deploy.local),
|
||||
slotProcessorActorName.next())
|
||||
(ActorProcessor[RequestContext, List[ProcessorOut]](actor), ())
|
||||
ActorProcessor[RequestContext, List[ProcessorOut]](actor)
|
||||
}.mapConcat(identity)
|
||||
}
|
||||
val split = b.add(Broadcast[ProcessorOut](2))
|
||||
|
|
|
|||
|
|
@ -179,7 +179,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape3(bypassInput, oneHundredContinue, applicationInput, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var requestStart: RequestStart = _
|
||||
|
||||
setHandler(bypassInput, new InHandler {
|
||||
|
|
@ -334,7 +334,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanOutShape2(in, httpOut, wsOut)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var target = httpOut
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
|
|
@ -362,7 +362,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape2(httpIn, wsIn, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
var websocketHandlerWasInstalled = false
|
||||
|
||||
setHandler(httpIn, conditionalTerminateInput(() ⇒ !websocketHandlerWasInstalled))
|
||||
|
|
@ -407,7 +407,7 @@ private[http] object HttpServerBluePrint {
|
|||
|
||||
override val shape = new FanInShape2(bytes, token, out)
|
||||
|
||||
override def createLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
passAlong(bytes, out, doFinish = true, doFail = true)
|
||||
passAlong(token, out, doFinish = false, doFail = true)
|
||||
setHandler(out, eagerTerminateOutput)
|
||||
|
|
|
|||
|
|
@ -7,12 +7,16 @@ package akka.http.impl.engine.ws
|
|||
import akka.http.impl.engine.ws.Protocol.Opcode
|
||||
import akka.util.ByteString
|
||||
|
||||
private[http] sealed trait FrameEventOrError
|
||||
|
||||
private[http] final case class FrameError(p: ProtocolException) extends FrameEventOrError
|
||||
|
||||
/**
|
||||
* The low-level Websocket framing model.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] sealed trait FrameEvent {
|
||||
private[http] sealed trait FrameEvent extends FrameEventOrError {
|
||||
def data: ByteString
|
||||
def lastPart: Boolean
|
||||
def withData(data: ByteString): FrameEvent
|
||||
|
|
|
|||
|
|
@ -4,11 +4,10 @@
|
|||
|
||||
package akka.http.impl.engine.ws
|
||||
|
||||
import akka.http.impl.util.{ ByteReader, ByteStringParserStage }
|
||||
import akka.stream.stage.{ StageState, SyncDirective, Context }
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import akka.stream.io.ByteStringParser
|
||||
import akka.stream.Attributes
|
||||
|
||||
/**
|
||||
* Streaming parser for the Websocket framing protocol as defined in RFC6455
|
||||
|
|
@ -36,108 +35,81 @@ import scala.annotation.tailrec
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] class FrameEventParser extends ByteStringParserStage[FrameEvent] {
|
||||
protected def onTruncation(ctx: Context[FrameEvent]): SyncDirective =
|
||||
ctx.fail(new ProtocolException("Data truncated"))
|
||||
private[http] object FrameEventParser extends ByteStringParser[FrameEvent] {
|
||||
import ByteStringParser._
|
||||
|
||||
def initial: StageState[ByteString, FrameEvent] = ReadFrameHeader
|
||||
override def createLogic(attr: Attributes) = new ParsingLogic {
|
||||
startWith(ReadFrameHeader)
|
||||
|
||||
object ReadFrameHeader extends ByteReadingState {
|
||||
def read(reader: ByteReader, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
import Protocol._
|
||||
|
||||
val flagsAndOp = reader.readByte()
|
||||
val maskAndLength = reader.readByte()
|
||||
|
||||
val flags = flagsAndOp & FLAGS_MASK
|
||||
val op = flagsAndOp & OP_MASK
|
||||
|
||||
val maskBit = (maskAndLength & MASK_MASK) != 0
|
||||
val length7 = maskAndLength & LENGTH_MASK
|
||||
|
||||
val length =
|
||||
length7 match {
|
||||
case 126 ⇒ reader.readShortBE().toLong
|
||||
case 127 ⇒ reader.readLongBE()
|
||||
case x ⇒ x.toLong
|
||||
}
|
||||
|
||||
if (length < 0) ctx.fail(new ProtocolException("Highest bit of 64bit length was set"))
|
||||
|
||||
val mask =
|
||||
if (maskBit) Some(reader.readIntBE())
|
||||
else None
|
||||
|
||||
def isFlagSet(mask: Int): Boolean = (flags & mask) != 0
|
||||
val header =
|
||||
FrameHeader(Opcode.forCode(op.toByte),
|
||||
mask,
|
||||
length,
|
||||
fin = isFlagSet(FIN_MASK),
|
||||
rsv1 = isFlagSet(RSV1_MASK),
|
||||
rsv2 = isFlagSet(RSV2_MASK),
|
||||
rsv3 = isFlagSet(RSV3_MASK))
|
||||
|
||||
val data = reader.remainingData
|
||||
val takeNow = (header.length min Int.MaxValue).toInt
|
||||
val thisFrameData = data.take(takeNow)
|
||||
val remaining = data.drop(takeNow)
|
||||
|
||||
val nextState =
|
||||
if (thisFrameData.length == length) ReadFrameHeader
|
||||
else readData(length - thisFrameData.length)
|
||||
|
||||
pushAndBecomeWithRemaining(FrameStart(header, thisFrameData.compact), nextState, remaining, ctx)
|
||||
}
|
||||
}
|
||||
|
||||
def readData(_remaining: Long): State =
|
||||
new State {
|
||||
var remaining = _remaining
|
||||
def onPush(elem: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
if (elem.size < remaining) {
|
||||
remaining -= elem.size
|
||||
ctx.push(FrameData(elem, lastPart = false))
|
||||
} else {
|
||||
require(remaining <= Int.MaxValue) // safe because, remaining <= elem.size <= Int.MaxValue
|
||||
val frameData = elem.take(remaining.toInt)
|
||||
val remainingData = elem.drop(remaining.toInt)
|
||||
|
||||
pushAndBecomeWithRemaining(FrameData(frameData.compact, lastPart = true), ReadFrameHeader, remainingData, ctx)
|
||||
}
|
||||
trait Step extends ParseStep[FrameEvent] {
|
||||
override def onTruncation(): Unit = failStage(new ProtocolException("Data truncated"))
|
||||
}
|
||||
|
||||
def becomeWithRemaining(nextState: State, remainingData: ByteString, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
become(nextState)
|
||||
nextState.onPush(remainingData, ctx)
|
||||
}
|
||||
def pushAndBecomeWithRemaining(elem: FrameEvent, nextState: State, remainingData: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
if (remainingData.isEmpty) {
|
||||
become(nextState)
|
||||
ctx.push(elem)
|
||||
} else {
|
||||
become(waitForPull(nextState, remainingData))
|
||||
ctx.push(elem)
|
||||
}
|
||||
object ReadFrameHeader extends Step {
|
||||
override def parse(reader: ByteReader): (FrameEvent, Step) = {
|
||||
import Protocol._
|
||||
|
||||
def waitForPull(nextState: State, remainingData: ByteString): State =
|
||||
new State {
|
||||
def onPush(elem: ByteString, ctx: Context[FrameEvent]): SyncDirective =
|
||||
throw new IllegalStateException("Mustn't push in this state")
|
||||
val flagsAndOp = reader.readByte()
|
||||
val maskAndLength = reader.readByte()
|
||||
|
||||
override def onPull(ctx: Context[FrameEvent]): SyncDirective = {
|
||||
become(nextState)
|
||||
nextState.onPush(remainingData, ctx)
|
||||
val flags = flagsAndOp & FLAGS_MASK
|
||||
val op = flagsAndOp & OP_MASK
|
||||
|
||||
val maskBit = (maskAndLength & MASK_MASK) != 0
|
||||
val length7 = maskAndLength & LENGTH_MASK
|
||||
|
||||
val length =
|
||||
length7 match {
|
||||
case 126 ⇒ reader.readShortBE().toLong
|
||||
case 127 ⇒ reader.readLongBE()
|
||||
case x ⇒ x.toLong
|
||||
}
|
||||
|
||||
if (length < 0) throw new ProtocolException("Highest bit of 64bit length was set")
|
||||
|
||||
val mask =
|
||||
if (maskBit) Some(reader.readIntBE())
|
||||
else None
|
||||
|
||||
def isFlagSet(mask: Int): Boolean = (flags & mask) != 0
|
||||
val header =
|
||||
FrameHeader(Opcode.forCode(op.toByte),
|
||||
mask,
|
||||
length,
|
||||
fin = isFlagSet(FIN_MASK),
|
||||
rsv1 = isFlagSet(RSV1_MASK),
|
||||
rsv2 = isFlagSet(RSV2_MASK),
|
||||
rsv3 = isFlagSet(RSV3_MASK))
|
||||
|
||||
val takeNow = (header.length min reader.remainingSize).toInt
|
||||
val thisFrameData = reader.take(takeNow)
|
||||
|
||||
val nextState =
|
||||
if (thisFrameData.length == length) ReadFrameHeader
|
||||
else new ReadData(length - thisFrameData.length)
|
||||
|
||||
(FrameStart(header, thisFrameData.compact), nextState)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object FrameEventParser {
|
||||
class ReadData(_remaining: Long) extends Step {
|
||||
var remaining = _remaining
|
||||
override def parse(reader: ByteReader): (FrameEvent, Step) =
|
||||
if (reader.remainingSize < remaining) {
|
||||
remaining -= reader.remainingSize
|
||||
(FrameData(reader.takeAll(), lastPart = false), this)
|
||||
} else {
|
||||
(FrameData(reader.take(remaining.toInt), lastPart = true), ReadFrameHeader)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def mask(bytes: ByteString, _mask: Option[Int]): ByteString =
|
||||
_mask match {
|
||||
case Some(m) ⇒ mask(bytes, m)._1
|
||||
case None ⇒ bytes
|
||||
}
|
||||
|
||||
def mask(bytes: ByteString, mask: Int): (ByteString, Int) = {
|
||||
@tailrec def rec(bytes: Array[Byte], offset: Int, mask: Int): Int =
|
||||
if (offset >= bytes.length) mask
|
||||
|
|
|
|||
|
|
@ -19,10 +19,10 @@ import scala.util.control.NonFatal
|
|||
*/
|
||||
private[http] object FrameHandler {
|
||||
|
||||
def create(server: Boolean): Flow[FrameEvent, Output, Unit] =
|
||||
Flow[FrameEvent].transform(() ⇒ new HandlerStage(server))
|
||||
def create(server: Boolean): Flow[FrameEventOrError, Output, Unit] =
|
||||
Flow[FrameEventOrError].transform(() ⇒ new HandlerStage(server))
|
||||
|
||||
private class HandlerStage(server: Boolean) extends StatefulStage[FrameEvent, Output] {
|
||||
private class HandlerStage(server: Boolean) extends StatefulStage[FrameEventOrError, Output] {
|
||||
type Ctx = Context[Output]
|
||||
def initial: State = Idle
|
||||
|
||||
|
|
@ -79,11 +79,6 @@ private[http] object FrameHandler {
|
|||
}
|
||||
}
|
||||
|
||||
private object Closed extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Ctx): SyncDirective =
|
||||
ctx.pull() // ignore
|
||||
}
|
||||
|
||||
private def becomeAndHandleWith(newState: State, part: FrameEvent)(implicit ctx: Ctx): SyncDirective = {
|
||||
become(newState)
|
||||
current.onPush(part, ctx)
|
||||
|
|
@ -132,7 +127,7 @@ private[http] object FrameHandler {
|
|||
}
|
||||
|
||||
private object CloseAfterPeerClosed extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Context[Output]): SyncDirective =
|
||||
def onPush(elem: FrameEventOrError, ctx: Context[Output]): SyncDirective =
|
||||
elem match {
|
||||
case FrameStart(FrameHeader(Opcode.Close, _, length, _, _, _, _), data) ⇒
|
||||
become(WaitForPeerTcpClose)
|
||||
|
|
@ -141,7 +136,7 @@ private[http] object FrameHandler {
|
|||
}
|
||||
}
|
||||
private object WaitForPeerTcpClose extends State {
|
||||
def onPush(elem: FrameEvent, ctx: Context[Output]): SyncDirective =
|
||||
def onPush(elem: FrameEventOrError, ctx: Context[Output]): SyncDirective =
|
||||
ctx.pull() // ignore
|
||||
}
|
||||
|
||||
|
|
@ -168,10 +163,11 @@ private[http] object FrameHandler {
|
|||
def handleFrameData(data: FrameData)(implicit ctx: Ctx): SyncDirective
|
||||
def handleFrameStart(start: FrameStart)(implicit ctx: Ctx): SyncDirective
|
||||
|
||||
def onPush(part: FrameEvent, ctx: Ctx): SyncDirective =
|
||||
def onPush(part: FrameEventOrError, ctx: Ctx): SyncDirective =
|
||||
part match {
|
||||
case data: FrameData ⇒ handleFrameData(data)(ctx)
|
||||
case start: FrameStart ⇒ handleFrameStart(start)(ctx)
|
||||
case FrameError(ex) ⇒ ctx.fail(ex)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -15,13 +15,19 @@ import akka.stream.stage.{ SyncDirective, Context, StatefulStage }
|
|||
* INTERNAL API
|
||||
*/
|
||||
private[http] object Masking {
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEvent, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
def apply(serverSide: Boolean, maskRandom: () ⇒ Random): BidiFlow[ /* net in */ FrameEvent, /* app out */ FrameEventOrError, /* app in */ FrameEvent, /* net out */ FrameEvent, Unit] =
|
||||
BidiFlow.fromFlowsMat(unmaskIf(serverSide), maskIf(!serverSide, maskRandom))(Keep.none)
|
||||
|
||||
def maskIf(condition: Boolean, maskRandom: () ⇒ Random): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
if (condition)
|
||||
Flow[FrameEvent]
|
||||
.transform(() ⇒ new Masking(maskRandom())) // new random per materialization
|
||||
.map {
|
||||
case f: FrameEvent ⇒ f
|
||||
case FrameError(ex) ⇒ throw ex
|
||||
}
|
||||
else Flow[FrameEvent]
|
||||
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEvent, Unit] =
|
||||
def unmaskIf(condition: Boolean): Flow[FrameEvent, FrameEventOrError, Unit] =
|
||||
if (condition) Flow[FrameEvent].transform(() ⇒ new Unmasking())
|
||||
else Flow[FrameEvent]
|
||||
|
||||
|
|
@ -41,19 +47,25 @@ private[http] object Masking {
|
|||
}
|
||||
|
||||
/** Implements both masking and unmasking which is mostly symmetric (because of XOR) */
|
||||
private abstract class Masker extends StatefulStage[FrameEvent, FrameEvent] {
|
||||
private abstract class Masker extends StatefulStage[FrameEvent, FrameEventOrError] {
|
||||
def extractMask(header: FrameHeader): Int
|
||||
def setNewMask(header: FrameHeader, mask: Int): FrameHeader
|
||||
|
||||
def initial: State = Idle
|
||||
|
||||
object Idle extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEvent]): SyncDirective =
|
||||
private object Idle extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective =
|
||||
part match {
|
||||
case start @ FrameStart(header, data) ⇒
|
||||
val mask = extractMask(header)
|
||||
become(new Running(mask))
|
||||
current.onPush(start.copy(header = setNewMask(header, mask)), ctx)
|
||||
try {
|
||||
val mask = extractMask(header)
|
||||
become(new Running(mask))
|
||||
current.onPush(start.copy(header = setNewMask(header, mask)), ctx)
|
||||
} catch {
|
||||
case p: ProtocolException ⇒
|
||||
become(Done)
|
||||
ctx.push(FrameError(p))
|
||||
}
|
||||
case _: FrameData ⇒
|
||||
ctx.fail(new IllegalStateException("unexpected FrameData (need FrameStart first)"))
|
||||
}
|
||||
|
|
@ -61,7 +73,7 @@ private[http] object Masking {
|
|||
private class Running(initialMask: Int) extends State {
|
||||
var mask = initialMask
|
||||
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEvent]): SyncDirective = {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective = {
|
||||
if (part.lastPart) become(Idle)
|
||||
|
||||
val (masked, newMask) = FrameEventParser.mask(part.data, mask)
|
||||
|
|
@ -69,5 +81,8 @@ private[http] object Masking {
|
|||
ctx.push(part.withData(data = masked))
|
||||
}
|
||||
}
|
||||
private object Done extends State {
|
||||
def onPush(part: FrameEvent, ctx: Context[FrameEventOrError]): SyncDirective = ctx.pull()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -40,12 +40,12 @@ private[http] object Websocket {
|
|||
/** The lowest layer that implements the binary protocol */
|
||||
def framing: BidiFlow[ByteString, FrameEvent, FrameEvent, ByteString, Unit] =
|
||||
BidiFlow.fromFlowsMat(
|
||||
Flow[ByteString].transform(() ⇒ new FrameEventParser),
|
||||
Flow[ByteString].via(FrameEventParser),
|
||||
Flow[FrameEvent].transform(() ⇒ new FrameEventRenderer))(Keep.none)
|
||||
.named("ws-framing")
|
||||
|
||||
/** The layer that handles masking using the rules defined in the specification */
|
||||
def masking(serverSide: Boolean, maskingRandomFactory: () ⇒ Random): BidiFlow[FrameEvent, FrameEvent, FrameEvent, FrameEvent, Unit] =
|
||||
def masking(serverSide: Boolean, maskingRandomFactory: () ⇒ Random): BidiFlow[FrameEvent, FrameEventOrError, FrameEvent, FrameEvent, Unit] =
|
||||
Masking(serverSide, maskingRandomFactory)
|
||||
.named("ws-masking")
|
||||
|
||||
|
|
@ -55,7 +55,7 @@ private[http] object Websocket {
|
|||
*/
|
||||
def frameHandling(serverSide: Boolean = true,
|
||||
closeTimeout: FiniteDuration,
|
||||
log: LoggingAdapter): BidiFlow[FrameEvent, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
log: LoggingAdapter): BidiFlow[FrameEventOrError, FrameHandler.Output, FrameOutHandler.Input, FrameStart, Unit] =
|
||||
BidiFlow.fromFlowsMat(
|
||||
FrameHandler.create(server = serverSide),
|
||||
FrameOutHandler.create(serverSide, closeTimeout, log))(Keep.none)
|
||||
|
|
@ -156,7 +156,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FanOutShape2(in, bypass, user)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
|
||||
setHandler(in, new InHandler {
|
||||
override def onPush(): Unit = {
|
||||
|
|
@ -187,7 +187,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FanInShape3(bypass, user, tick, out)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
|
||||
passAlong(bypass, out, doFinish = true, doFail = true)
|
||||
passAlong(user, out, doFinish = false, doFail = false)
|
||||
|
|
@ -210,7 +210,7 @@ private[http] object Websocket {
|
|||
|
||||
val shape = new FlowShape(in, out)
|
||||
|
||||
def createLogic = new GraphStageLogic(shape) {
|
||||
def createLogic(effectiveAttributes: Attributes) = new GraphStageLogic(shape) {
|
||||
setHandler(out, new OutHandler {
|
||||
override def onPull(): Unit = pull(in)
|
||||
})
|
||||
|
|
|
|||
|
|
@ -21,17 +21,17 @@ private[akka] class ByteReader(input: ByteString) {
|
|||
|
||||
def currentOffset: Int = off
|
||||
def remainingData: ByteString = input.drop(off)
|
||||
def fromStartToHere: ByteString = input.take(currentOffset)
|
||||
def fromStartToHere: ByteString = input.take(off)
|
||||
|
||||
def readByte(): Int =
|
||||
if (off < input.length) {
|
||||
val x = input(off)
|
||||
off += 1
|
||||
x.toInt & 0xFF
|
||||
x & 0xFF
|
||||
} else throw NeedMoreData
|
||||
def readShortLE(): Int = readByte() | (readByte() << 8)
|
||||
def readIntLE(): Int = readShortLE() | (readShortLE() << 16)
|
||||
def readLongLE(): Long = (readIntBE() & 0xffffffffL) | ((readIntLE() & 0xffffffffL) << 32)
|
||||
def readLongLE(): Long = (readIntLE() & 0xffffffffL) | ((readIntLE() & 0xffffffffL) << 32)
|
||||
|
||||
def readShortBE(): Int = (readByte() << 8) | readByte()
|
||||
def readIntBE(): Int = (readShortBE() << 16) | readShortBE()
|
||||
|
|
|
|||
|
|
@ -128,7 +128,7 @@ package object util {
|
|||
package util {
|
||||
|
||||
import akka.http.scaladsl.model.{ ContentType, HttpEntity }
|
||||
import akka.stream.{ Outlet, Inlet, FlowShape }
|
||||
import akka.stream.{ Attributes, Outlet, Inlet, FlowShape }
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
private[http] class ToStrict(timeout: FiniteDuration, contentType: ContentType)
|
||||
|
|
@ -138,7 +138,7 @@ package util {
|
|||
val out = Outlet[HttpEntity.Strict]("out")
|
||||
override val shape = FlowShape(in, out)
|
||||
|
||||
override def createLogic: GraphStageLogic = new GraphStageLogic(shape) {
|
||||
override def createLogic(inheritedAttributes: Attributes): GraphStageLogic = new TimerGraphStageLogic(shape) {
|
||||
var bytes = ByteString.newBuilder
|
||||
private var emptyStream = false
|
||||
|
||||
|
|
|
|||
|
|
@ -75,7 +75,7 @@ sealed trait HttpEntity extends jm.HttpEntity {
|
|||
def withContentType(contentType: ContentType): HttpEntity
|
||||
|
||||
/** Java API */
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, _] = stream.javadsl.Source.fromGraph(dataBytes)
|
||||
def getDataBytes: stream.javadsl.Source[ByteString, AnyRef] = stream.javadsl.Source.fromGraph(dataBytes.asInstanceOf[Source[ByteString, AnyRef]])
|
||||
|
||||
/** Java API */
|
||||
def getContentLengthOption: japi.Option[JLong] =
|
||||
|
|
@ -147,9 +147,11 @@ object HttpEntity {
|
|||
def apply(contentType: ContentType, data: Source[ByteString, Any]): Chunked =
|
||||
Chunked.fromData(contentType, data)
|
||||
|
||||
def apply(contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): UniversalEntity = {
|
||||
def apply(contentType: ContentType, file: File, chunkSize: Int = -1): UniversalEntity = {
|
||||
val fileLength = file.length
|
||||
if (fileLength > 0) Default(contentType, fileLength, SynchronousFileSource(file, chunkSize))
|
||||
if (fileLength > 0)
|
||||
Default(contentType, fileLength,
|
||||
if (chunkSize > 0) SynchronousFileSource(file, chunkSize) else SynchronousFileSource(file))
|
||||
else empty(contentType)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -194,7 +194,7 @@ object Multipart {
|
|||
* To create an instance with several parts or for multiple files, use
|
||||
* ``FormData(BodyPart.fromFile("field1", ...), BodyPart.fromFile("field2", ...)``
|
||||
*/
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): FormData =
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = -1): FormData =
|
||||
FormData(Source.single(BodyPart.fromFile(name, contentType, file, chunkSize)))
|
||||
|
||||
/**
|
||||
|
|
@ -237,7 +237,7 @@ object Multipart {
|
|||
/**
|
||||
* Creates a BodyPart backed by a File that will be streamed using a SynchronousFileSource.
|
||||
*/
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = SynchronousFileSource.DefaultChunkSize): BodyPart =
|
||||
def fromFile(name: String, contentType: ContentType, file: File, chunkSize: Int = -1): BodyPart =
|
||||
BodyPart(name, HttpEntity(contentType, file, chunkSize), Map("filename" -> file.getName))
|
||||
|
||||
def unapply(value: BodyPart): Option[(String, BodyPartEntity, Map[String, String], immutable.Seq[HttpHeader])] =
|
||||
|
|
|
|||
|
|
@ -1,4 +1,5 @@
|
|||
akka {
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
actor {
|
||||
serialize-creators = on
|
||||
serialize-messages = on
|
||||
|
|
|
|||
|
|
@ -57,8 +57,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
|
||||
requestIn.sendNext(HttpRequest(uri = "/") -> 42)
|
||||
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(response), 42) = responseOut.expectNext()
|
||||
response.headers should contain(RawHeader("Req-Host", s"$serverHostName:$serverPort"))
|
||||
}
|
||||
|
|
@ -116,8 +116,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
val (requestIn, responseOut, responseOutSub, hcp) = cachedHostConnectionPool[Int]()
|
||||
|
||||
requestIn.sendNext(HttpRequest(uri = "/a") -> 42)
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(response1), 42) = responseOut.expectNext()
|
||||
connNr(response1) shouldEqual 1
|
||||
|
||||
|
|
@ -222,8 +222,8 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
|
||||
requestIn.sendNext(HttpRequest(uri = "/") -> 42)
|
||||
|
||||
acceptIncomingConnection()
|
||||
responseOutSub.request(1)
|
||||
acceptIncomingConnection()
|
||||
val (Success(_), 42) = responseOut.expectNext()
|
||||
}
|
||||
}
|
||||
|
|
@ -346,7 +346,7 @@ class ConnectionPoolSpec extends AkkaSpec("""
|
|||
def flowTestBench[T, Mat](poolFlow: Flow[(HttpRequest, T), (Try[HttpResponse], T), Mat]) = {
|
||||
val requestIn = TestPublisher.probe[(HttpRequest, T)]()
|
||||
val responseOut = TestSubscriber.manualProbe[(Try[HttpResponse], T)]
|
||||
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).toMat(Sink(responseOut))(Keep.left).run()
|
||||
val hcp = Source(requestIn).viaMat(poolFlow)(Keep.right).to(Sink(responseOut)).run()
|
||||
val responseOutSub = responseOut.expectSubscription()
|
||||
(requestIn, responseOut, responseOutSub, hcp)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
@ -228,7 +228,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
@ -254,7 +254,7 @@ class LowLevelOutgoingConnectionSpec extends AkkaSpec("akka.loggers = []\n akka.
|
|||
|
|
||||
|""")
|
||||
val sub = probe.expectSubscription()
|
||||
sub.expectRequest(4)
|
||||
sub.expectRequest()
|
||||
sub.sendNext(ByteString("ABC"))
|
||||
expectWireData("ABC")
|
||||
sub.sendNext(ByteString("DEF"))
|
||||
|
|
|
|||
|
|
@ -6,20 +6,18 @@ package akka.http.impl.engine.client
|
|||
|
||||
import org.scalatest.concurrent.PatienceConfiguration.Timeout
|
||||
import org.scalatest.concurrent.ScalaFutures
|
||||
|
||||
import akka.stream.ActorMaterializer
|
||||
import akka.stream.io._
|
||||
import akka.stream.scaladsl._
|
||||
import akka.stream.testkit.AkkaSpec
|
||||
|
||||
import akka.http.impl.util._
|
||||
|
||||
import akka.http.scaladsl.{ HttpsContext, Http }
|
||||
import akka.http.scaladsl.model.{ StatusCodes, HttpResponse, HttpRequest }
|
||||
import akka.http.scaladsl.model.headers.Host
|
||||
import org.scalatest.time.{ Span, Seconds }
|
||||
|
||||
import scala.concurrent.Future
|
||||
import akka.testkit.EventFilter
|
||||
import javax.net.ssl.SSLException
|
||||
|
||||
class TlsEndpointVerificationSpec extends AkkaSpec("""
|
||||
#akka.loggers = []
|
||||
|
|
@ -30,7 +28,7 @@ class TlsEndpointVerificationSpec extends AkkaSpec("""
|
|||
val timeout = Timeout(Span(3, Seconds))
|
||||
|
||||
"The client implementation" should {
|
||||
"not accept certificates signed by unknown CA" in {
|
||||
"not accept certificates signed by unknown CA" in EventFilter[SSLException](occurrences = 1).intercept {
|
||||
val pipe = pipeline(Http().defaultClientHttpsContext, hostname = "akka.example.org") // default context doesn't include custom CA
|
||||
|
||||
whenReady(pipe(HttpRequest(uri = "https://akka.example.org/")).failed, timeout) { e ⇒
|
||||
|
|
|
|||
|
|
@ -307,13 +307,12 @@ class FramingSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
}
|
||||
|
||||
private def parseToEvents(bytes: Seq[ByteString]): immutable.Seq[FrameEvent] =
|
||||
Source(bytes.toVector).transform(newParser).runFold(Vector.empty[FrameEvent])(_ :+ _)
|
||||
Source(bytes.toVector).via(FrameEventParser).runFold(Vector.empty[FrameEvent])(_ :+ _)
|
||||
.awaitResult(1.second)
|
||||
private def renderToByteString(events: immutable.Seq[FrameEvent]): ByteString =
|
||||
Source(events).transform(newRenderer).runFold(ByteString.empty)(_ ++ _)
|
||||
.awaitResult(1.second)
|
||||
|
||||
protected def newParser(): Stage[ByteString, FrameEvent] = new FrameEventParser
|
||||
protected def newRenderer(): Stage[FrameEvent, ByteString] = new FrameEventRenderer
|
||||
|
||||
import scala.language.implicitConversions
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import akka.stream.testkit._
|
|||
import akka.util.ByteString
|
||||
import akka.http.scaladsl.model.ws._
|
||||
import Protocol.Opcode
|
||||
import akka.testkit.EventFilter
|
||||
|
||||
class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
||||
import WSTestUtils._
|
||||
|
|
@ -595,15 +596,18 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
netIn.expectCancellation()
|
||||
}
|
||||
"if user handler fails" in new ServerTestSetup {
|
||||
messageOut.sendError(new RuntimeException("Oops, user handler failed!"))
|
||||
expectCloseCodeOnNetwork(Protocol.CloseCodes.UnexpectedCondition)
|
||||
EventFilter[RuntimeException](message = "Oops, user handler failed!", occurrences = 1)
|
||||
.intercept {
|
||||
messageOut.sendError(new RuntimeException("Oops, user handler failed!"))
|
||||
expectCloseCodeOnNetwork(Protocol.CloseCodes.UnexpectedCondition)
|
||||
|
||||
expectNoNetworkData() // wait for peer to close regularly
|
||||
pushInput(closeFrame(Protocol.CloseCodes.Regular, mask = true))
|
||||
expectNoNetworkData() // wait for peer to close regularly
|
||||
pushInput(closeFrame(Protocol.CloseCodes.Regular, mask = true))
|
||||
|
||||
expectComplete(messageIn)
|
||||
netOut.expectComplete()
|
||||
netIn.expectCancellation()
|
||||
expectComplete(messageIn)
|
||||
netOut.expectComplete()
|
||||
netIn.expectCancellation()
|
||||
}
|
||||
}
|
||||
"if peer closes with invalid close frame" - {
|
||||
"close code outside of the valid range" in new ServerTestSetup {
|
||||
|
|
@ -828,7 +832,7 @@ class MessageSpec extends FreeSpec with Matchers with WithMaterializerSpec {
|
|||
|
||||
Source(netIn)
|
||||
.via(printEvent("netIn"))
|
||||
.transform(() ⇒ new FrameEventParser)
|
||||
.via(FrameEventParser)
|
||||
.via(Websocket.stack(serverSide, maskingRandomFactory = Randoms.SecureRandomInstances, closeTimeout = closeTimeout, log = system.log).join(messageHandler))
|
||||
.via(printEvent("frameRendererIn"))
|
||||
.transform(() ⇒ new FrameEventRenderer)
|
||||
|
|
|
|||
|
|
@ -24,8 +24,8 @@ import akka.http.scaladsl.model.HttpMethods._
|
|||
import akka.http.scaladsl.model._
|
||||
import akka.http.scaladsl.model.headers._
|
||||
import akka.http.impl.util._
|
||||
|
||||
import scala.util.{ Failure, Try, Success }
|
||||
import java.net.BindException
|
||||
|
||||
class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
|
|
@ -56,7 +56,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
sub.cancel()
|
||||
}
|
||||
|
||||
"report failure if bind fails" in {
|
||||
"report failure if bind fails" in EventFilter[BindException](occurrences = 2).intercept {
|
||||
val (_, hostname, port) = TestUtils.temporaryServerHostnameAndPort()
|
||||
val binding = Http().bind(hostname, port)
|
||||
val probe1 = TestSubscriber.manualProbe[Http.IncomingConnection]()
|
||||
|
|
@ -170,7 +170,8 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
|
|||
// waiting for the timeout to happen on the client
|
||||
intercept[StreamTcpException] { Await.result(clientsResponseFuture, 2.second) }
|
||||
|
||||
(System.nanoTime() - serverReceivedRequestAtNanos).millis should be >= theIdleTimeout
|
||||
val fudge = 100.millis
|
||||
((System.nanoTime() - serverReceivedRequestAtNanos).nanos + fudge) should be >= theIdleTimeout
|
||||
}
|
||||
|
||||
"log materialization errors in `bindAndHandle`" which {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue