!str #16902: Unify stream internal representation

also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
Endre Sándor Varga 2015-01-28 14:19:50 +01:00
parent cac9c9f2fb
commit 8d77fa8b29
230 changed files with 7814 additions and 9596 deletions

View file

@ -42,19 +42,19 @@ public final class HttpEntities {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file);
}
public static HttpEntityDefault create(ContentType contentType, long contentLength, Source<ByteString> data) {
public static HttpEntityDefault create(ContentType contentType, long contentLength, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data);
}
public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Source<ByteString> data) {
public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data);
}
public static HttpEntityIndefiniteLength createIndefiniteLength(ContentType contentType, Source<ByteString> data) {
public static HttpEntityIndefiniteLength createIndefiniteLength(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return new akka.http.model.HttpEntity.IndefiniteLength((akka.http.model.ContentType) contentType, data);
}
public static HttpEntityChunked createChunked(ContentType contentType, Source<ByteString> data) {
public static HttpEntityChunked createChunked(ContentType contentType, Source<ByteString, scala.runtime.BoxedUnit> data) {
return akka.http.model.HttpEntity.Chunked$.MODULE$.fromData(
(akka.http.model.ContentType) contentType,
data);

View file

@ -73,5 +73,5 @@ public interface HttpEntity {
/**
* Returns a stream of data bytes this entity consists of.
*/
public abstract Source<ByteString> getDataBytes();
public abstract Source<ByteString, scala.Unit> getDataBytes();
}

View file

@ -11,5 +11,5 @@ import akka.stream.scaladsl.Source;
* stream of {@link ChunkStreamPart}.
*/
public abstract class HttpEntityChunked implements RequestEntity, ResponseEntity {
public abstract Source<ChunkStreamPart> getChunks();
public abstract Source<ChunkStreamPart, scala.Unit> getChunks();
}

View file

@ -13,5 +13,5 @@ import akka.stream.scaladsl.Source;
* available for Http responses.
*/
public abstract class HttpEntityCloseDelimited implements ResponseEntity {
public abstract Source<ByteString> data();
public abstract Source<ByteString, scala.Unit> data();
}

View file

@ -12,5 +12,5 @@ import akka.stream.scaladsl.Source;
*/
public abstract class HttpEntityDefault implements BodyPartEntity, RequestEntity, ResponseEntity {
public abstract long contentLength();
public abstract Source<ByteString> data();
public abstract Source<ByteString, scala.Unit> data();
}

View file

@ -11,5 +11,5 @@ import akka.stream.scaladsl.Source;
* Represents an entity without a predetermined content-length to use in a BodyParts.
*/
public abstract class HttpEntityIndefiniteLength implements BodyPartEntity {
public abstract Source<ByteString> data();
public abstract Source<ByteString, scala.Unit> data();
}

View file

@ -27,12 +27,12 @@ public abstract class Util {
@SuppressWarnings("unchecked") // no support for covariance of Publisher in Java
// needed to provide covariant conversions that the Java interfaces don't provide automatically.
// The alternative would be having to cast around everywhere instead of doing it here in a central place.
public static <U, T extends U> Source<U> convertPublisher(Source<T> p) {
return (Source<U>)(Source) p;
public static <U, T extends U> Source<U, scala.Unit> convertPublisher(Source<T, scala.Unit> p) {
return (Source<U, scala.Unit>)(Source) p;
}
@SuppressWarnings("unchecked")
public static <T, U extends T> Source<U> upcastSource(Source<T> p) {
return (Source<U>)(Source) p;
public static <T, U extends T> Source<U, scala.Unit> upcastSource(Source<T, scala.Unit> p) {
return (Source<U, scala.Unit>)(Source) p;
}
@SuppressWarnings("unchecked")
public static scala.collection.immutable.Map<String, String> convertMapToScala(Map<String, String> map) {

View file

@ -5,13 +5,15 @@
package akka.http
import java.net.InetSocketAddress
import akka.http.engine.server.HttpServer.HttpServerPorts
import akka.stream.Graph
import com.typesafe.config.Config
import scala.collection.immutable
import scala.concurrent.Future
import akka.event.LoggingAdapter
import akka.util.ByteString
import akka.io.Inet
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._
import akka.http.engine.client.{ HttpClient, ClientConnectionSettings }
import akka.http.engine.server.{ HttpServer, ServerSettings }
@ -27,32 +29,93 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
def bind(interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log): ServerBinding = {
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Source[IncomingConnection, Future[ServerBinding]] = {
val endpoint = new InetSocketAddress(interface, port)
val effectiveSettings = ServerSettings(settings)
val tcpBinding = StreamTcp().bind(endpoint, backlog, options, effectiveSettings.timeouts.idleTimeout)
new ServerBinding {
def localAddress(mm: MaterializedMap): Future[InetSocketAddress] = tcpBinding.localAddress(mm)
val connections = tcpBinding.connections map { tcpConn
new IncomingConnection {
def localAddress = tcpConn.localAddress
def remoteAddress = tcpConn.remoteAddress
def handleWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer) =
tcpConn.handleWith(HttpServer.serverFlowToTransport(handler, effectiveSettings, log))
}
val connections: Source[StreamTcp.IncomingConnection, Future[StreamTcp.ServerBinding]] = StreamTcp().bind(endpoint, backlog, options, effectiveSettings.timeouts.idleTimeout)
val serverBlueprint: Graph[HttpServerPorts, Unit] = HttpServer.serverBlueprint(effectiveSettings, log)
connections.map { conn
val flow = Flow(conn.flow, serverBlueprint)(Keep.right) { implicit b
(tcp, http)
import FlowGraph.Implicits._
tcp.outlet ~> http.bytesIn
http.bytesOut ~> tcp.inlet
(http.httpResponses, http.httpRequests)
}
def unbind(mm: MaterializedMap): Future[Unit] = tcpBinding.unbind(mm)
IncomingConnection(conn.localAddress, conn.remoteAddress, flow)
}.mapMaterialized { tcpBindingFuture
import system.dispatcher
tcpBindingFuture.map { tcpBinding ServerBinding(tcpBinding.localAddress)(() tcpBinding.unbind()) }
}
}
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def bindAndstartHandlingWith(handler: Flow[HttpRequest, HttpResponse, _],
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] = {
bind(interface, port, backlog, options, settings, log).toMat(Sink.foreach { conn
conn.flow.join(handler)
})(Keep.left).run()
}
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def bindAndStartHandlingWithSyncHandler(handler: HttpRequest HttpResponse,
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] =
bindAndstartHandlingWith(Flow[HttpRequest].map(handler), interface, port, backlog, options, settings, log)
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithAsyncHandler(handler: HttpRequest Future[HttpResponse],
interface: String, port: Int = 80, backlog: Int = 100,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit fm: ActorFlowMaterializer): Future[ServerBinding] =
bindAndstartHandlingWith(Flow[HttpRequest].mapAsync(handler), interface, port, backlog, options, settings, log)
/**
* Transforms a given HTTP-level server [[Flow]] into a lower-level TCP transport flow.
*/
def serverFlowToTransport(serverFlow: Flow[HttpRequest, HttpResponse],
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit mat: FlowMaterializer): Flow[ByteString, ByteString] = {
def serverFlowToTransport[Mat](serverFlow: Flow[HttpRequest, HttpResponse, Mat],
settings: Option[ServerSettings] = None,
log: LoggingAdapter = system.log)(implicit mat: ActorFlowMaterializer): Flow[ByteString, ByteString, Mat] = {
val effectiveSettings = ServerSettings(settings)
HttpServer.serverFlowToTransport(serverFlow, effectiveSettings, log)
val serverBlueprint: Graph[HttpServerPorts, Unit] = HttpServer.serverBlueprint(effectiveSettings, log)
Flow(serverBlueprint, serverFlow)(Keep.right) { implicit b
(server, user)
import FlowGraph.Implicits._
server.httpRequests ~> user.inlet
user.outlet ~> server.httpResponses
(server.bytesIn, server.bytesOut)
}
}
/**
@ -62,27 +125,46 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
localAddress: Option[InetSocketAddress] = None,
options: immutable.Traversable[Inet.SocketOption] = Nil,
settings: Option[ClientConnectionSettings] = None,
log: LoggingAdapter = system.log): OutgoingConnection = {
log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse, Future[OutgoingConnection]] = {
val effectiveSettings = ClientConnectionSettings(settings)
val remoteAddr = new InetSocketAddress(host, port)
val transportFlow = StreamTcp().outgoingConnection(remoteAddr, localAddress,
options, effectiveSettings.connectingTimeout, effectiveSettings.idleTimeout)
new OutgoingConnection {
def remoteAddress = remoteAddr
def localAddress(mm: MaterializedMap) = transportFlow.localAddress(mm)
val flow = HttpClient.transportToConnectionClientFlow(transportFlow.flow, remoteAddr, effectiveSettings, log)
val clientBluePrint = HttpClient.clientBlueprint(remoteAddr, effectiveSettings, log)
Flow(transportFlow, clientBluePrint)(Keep.left) { implicit b
(tcp, client)
import FlowGraph.Implicits._
tcp.outlet ~> client.bytesIn
client.bytesOut ~> tcp.inlet
(client.httpRequests, client.httpResponses)
}.mapMaterialized { tcpConnFuture
import system.dispatcher
tcpConnFuture.map { tcpConn OutgoingConnection(tcpConn.localAddress, tcpConn.remoteAddress) }
}
}
/**
* Transforms the given low-level TCP client transport [[Flow]] into a higher-level HTTP client flow.
*/
def transportToConnectionClientFlow(transport: Flow[ByteString, ByteString],
remoteAddress: InetSocketAddress, // TODO: removed after #16168 is cleared
settings: Option[ClientConnectionSettings] = None,
log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse] = {
def transportToConnectionClientFlow[Mat](transport: Flow[ByteString, ByteString, Mat],
remoteAddress: InetSocketAddress, // TODO: removed after #16168 is cleared
settings: Option[ClientConnectionSettings] = None,
log: LoggingAdapter = system.log): Flow[HttpRequest, HttpResponse, Mat] = {
val effectiveSettings = ClientConnectionSettings(settings)
HttpClient.transportToConnectionClientFlow(transport, remoteAddress, effectiveSettings, log)
val clientBlueprint = HttpClient.clientBlueprint(remoteAddress, effectiveSettings, log)
Flow(clientBlueprint, transport)(Keep.right) { implicit b
(client, tcp)
import FlowGraph.Implicits._
client.bytesOut ~> tcp.inlet
tcp.outlet ~> client.bytesIn
(client.httpRequests, client.httpResponses)
}
}
}
@ -90,119 +172,57 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
/**
* Represents a prospective HTTP server binding.
*
* @param localAddress The local address of the endpoint bound by the materialization of the `connections` [[Source]]
*
*/
sealed trait ServerBinding {
/**
* The local address of the endpoint bound by the materialization of the `connections` [[Source]]
* whose [[MaterializedMap]] is passed as parameter.
*/
def localAddress(materializedMap: MaterializedMap): Future[InetSocketAddress]
/**
* The stream of accepted incoming connections.
* Can be materialized several times but only one subscription can be "live" at one time, i.e.
* subsequent materializations will reject subscriptions with an [[StreamTcp.BindFailedException]] if the previous
* materialization still has an uncancelled subscription.
* Cancelling the subscription to a materialization of this source will cause the listening port to be unbound.
*/
def connections: Source[IncomingConnection]
case class ServerBinding(localAddress: InetSocketAddress)(private val unbindAction: () Future[Unit]) {
/**
* Asynchronously triggers the unbinding of the port that was bound by the materialization of the `connections`
* [[Source]] whose [[MaterializedMap]] is passed as parameter.
* [[Source]]
*
* The produced [[Future]] is fulfilled when the unbinding has been completed.
*/
def unbind(materializedMap: MaterializedMap): Future[Unit]
def unbind(): Future[Unit] = unbindAction()
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap =
connections.to(ForeachSink(_ handleWith handler)).run()
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: FlowMaterializer): MaterializedMap =
startHandlingWith(Flow[HttpRequest].map(handler))
/**
* Materializes the `connections` [[Source]] and handles all connections with the given flow.
*
* Note that there is no backpressure being applied to the `connections` [[Source]], i.e. all
* connections are being accepted at maximum rate, which, depending on the applications, might
* present a DoS risk!
*/
def startHandlingWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap =
startHandlingWith(Flow[HttpRequest].mapAsync(handler))
}
/**
* Represents one accepted incoming HTTP connection.
*/
sealed trait IncomingConnection {
/**
* The local address this connection is bound to.
*/
def localAddress: InetSocketAddress
/**
* The remote address this connection is bound to.
*/
def remoteAddress: InetSocketAddress
case class IncomingConnection(
localAddress: InetSocketAddress,
remoteAddress: InetSocketAddress,
flow: Flow[HttpResponse, HttpRequest, Unit]) {
/**
* Handles the connection with the given flow, which is materialized exactly once
* and the respective [[MaterializedMap]] returned.
*/
def handleWith(handler: Flow[HttpRequest, HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap
def handleWith[Mat](handler: Flow[HttpRequest, HttpResponse, Mat])(implicit fm: ActorFlowMaterializer): Mat =
flow.join(handler).mapMaterialized(_._2).run()
/**
* Handles the connection with the given handler function.
* Returns the [[MaterializedMap]] of the underlying flow materialization.
*/
def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: FlowMaterializer): MaterializedMap =
def handleWithSyncHandler(handler: HttpRequest HttpResponse)(implicit fm: ActorFlowMaterializer): Unit =
handleWith(Flow[HttpRequest].map(handler))
/**
* Handles the connection with the given handler function.
* Returns the [[MaterializedMap]] of the underlying flow materialization.
*/
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: FlowMaterializer): MaterializedMap =
def handleWithAsyncHandler(handler: HttpRequest Future[HttpResponse])(implicit fm: ActorFlowMaterializer): Unit =
handleWith(Flow[HttpRequest].mapAsync(handler))
}
/**
* Represents a prospective outgoing HTTP connection.
*/
sealed trait OutgoingConnection {
/**
* The remote address this connection is or will be bound to.
*/
def remoteAddress: InetSocketAddress
case class OutgoingConnection(localAddress: InetSocketAddress, remoteAddress: InetSocketAddress) {
/**
* The local address of the endpoint bound by the materialization of the connection materialization
* whose [[MaterializedMap]] is passed as parameter.
*/
def localAddress(mMap: MaterializedMap): Future[InetSocketAddress]
/**
* A flow representing the HTTP server on a single HTTP connection.
* This flow can be materialized several times, every materialization will open a new connection to the `remoteAddress`.
* If the connection cannot be established the materialized stream will immediately be terminated
* with a [[akka.stream.StreamTcpException]].
*/
def flow: Flow[HttpRequest, HttpResponse]
}
//////////////////// EXTENSION SETUP ///////////////////

View file

@ -5,12 +5,14 @@
package akka.http.engine.client
import java.net.InetSocketAddress
import scala.annotation.tailrec
import scala.collection.immutable.Seq
import scala.collection.mutable.ListBuffer
import akka.stream.stage._
import akka.util.ByteString
import akka.event.LoggingAdapter
import akka.stream.FlattenStrategy
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._
import akka.http.model.{ IllegalResponseException, HttpMethod, HttpRequest, HttpResponse }
@ -23,10 +25,35 @@ import akka.http.util._
*/
private[http] object HttpClient {
def transportToConnectionClientFlow(transport: Flow[ByteString, ByteString],
remoteAddress: InetSocketAddress,
settings: ClientConnectionSettings,
log: LoggingAdapter): Flow[HttpRequest, HttpResponse] = {
case class HttpClientPorts(
bytesIn: Inlet[ByteString],
bytesOut: Outlet[ByteString],
httpRequests: Inlet[HttpRequest],
httpResponses: Outlet[HttpResponse]) extends Shape {
override val inlets: Seq[Inlet[_]] = bytesIn :: httpRequests :: Nil
override val outlets: Seq[Outlet[_]] = bytesOut :: httpResponses :: Nil
override def deepCopy(): Shape = HttpClientPorts(
new Inlet(bytesIn.toString),
new Outlet(bytesOut.toString),
new Inlet(httpResponses.toString),
new Outlet(httpRequests.toString))
override def copyFromPorts(inlets: Seq[Inlet[_]], outlets: Seq[Outlet[_]]): Shape = {
val bIn :: htpIn :: Nil = inlets
val bOut :: htpOut :: Nil = outlets
HttpClientPorts(
bIn.asInstanceOf[Inlet[ByteString]],
bOut.asInstanceOf[Outlet[ByteString]],
htpIn.asInstanceOf[Inlet[HttpRequest]],
htpOut.asInstanceOf[Outlet[HttpResponse]])
}
}
def clientBlueprint(remoteAddress: InetSocketAddress,
settings: ClientConnectionSettings,
log: LoggingAdapter): Graph[HttpClientPorts, Unit] = {
import settings._
// the initial header parser we initially use for every connection,
@ -57,24 +84,11 @@ private[http] object HttpClient {
+------------+
*/
val requestIn = UndefinedSource[HttpRequest]
val responseOut = UndefinedSink[HttpResponse]
val methodBypassFanout = Broadcast[HttpRequest]
val responseParsingMerge = new ResponseParsingMerge(rootParser)
val terminationFanout = Broadcast[HttpResponse]
val terminationMerge = new TerminationMerge
val requestRendering = Flow[HttpRequest]
val requestRendering: Flow[HttpRequest, ByteString, Unit] = Flow[HttpRequest]
.map(RequestRenderingContext(_, remoteAddress))
.section(name("renderer"))(_.transform(() requestRendererFactory.newRenderer))
.flatten(FlattenStrategy.concat)
val transportFlow = Flow[ByteString]
.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing request stream error")))
.via(transport)
val methodBypass = Flow[HttpRequest].map(_.method)
import ParserOutput._
@ -89,34 +103,42 @@ private[http] object HttpClient {
case (MessageStartError(_, info), _) throw IllegalResponseException(info)
}
import FlowGraphImplicits._
FlowGraph.partial() { implicit b
import FlowGraph.Implicits._
val methodBypassFanout = b.add(Broadcast[HttpRequest](2))
val responseParsingMerge = b.add(new ResponseParsingMerge(rootParser))
Flow() { implicit b
requestIn ~> methodBypassFanout ~> terminationMerge.requestInput ~> requestRendering ~> transportFlow ~>
responseParsingMerge.dataInput ~> responsePrep ~> terminationFanout ~> responseOut
methodBypassFanout ~> methodBypass ~> responseParsingMerge.methodBypassInput
terminationFanout ~> terminationMerge.terminationBackchannelInput
val terminationFanout = b.add(Broadcast[HttpResponse](2))
val terminationMerge = b.add(new TerminationMerge)
b.allowCycles()
val bytesOut = (terminationMerge.out ~>
requestRendering.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing request stream error")))).outlet
requestIn -> responseOut
val bytesIn = responseParsingMerge.in0
methodBypassFanout.out(0) ~> terminationMerge.in0
methodBypassFanout.out(1) ~> methodBypass ~> responseParsingMerge.in1
responseParsingMerge.out ~> responsePrep ~> terminationFanout.in
terminationFanout.out(0) ~> terminationMerge.in1
HttpClientPorts(bytesIn, bytesOut, methodBypassFanout.in, terminationFanout.out(1))
}
}
// a simple merge stage that simply forwards its first input and ignores its second input
// (the terminationBackchannelInput), but applies a special completion handling
class TerminationMerge extends FlexiMerge[HttpRequest] {
class TerminationMerge
extends FlexiMerge[HttpRequest, FanInShape2[HttpRequest, HttpResponse, HttpRequest]](new FanInShape2("TerminationMerge"), OperationAttributes.name("TerminationMerge")) {
import FlexiMerge._
val requestInput = createInputPort[HttpRequest]()
val terminationBackchannelInput = createInputPort[HttpResponse]()
def createMergeLogic() = new MergeLogic[HttpRequest] {
override def inputHandles(inputCount: Int) = {
require(inputCount == 2, s"TerminationMerge must have 2 connected inputs, was $inputCount")
Vector(requestInput, terminationBackchannelInput)
}
def createMergeLogic(p: PortT) = new MergeLogic[HttpRequest] {
override def initialState = State[Any](ReadAny(requestInput, terminationBackchannelInput)) {
val requestInput = p.in0
val terminationBackchannelInput = p.in1
override def initialState = State[Any](ReadAny(p)) {
case (ctx, _, request: HttpRequest) { ctx.emit(request); SameState }
case _ SameState // simply drop all responses, we are only interested in the completion of the response input
}
@ -140,22 +162,17 @@ private[http] object HttpClient {
* 2. Read from the dataInput until exactly one response has been fully received
* 3. Go back to 1.
*/
class ResponseParsingMerge(rootParser: HttpResponseParser) extends FlexiMerge[List[ResponseOutput]] {
class ResponseParsingMerge(rootParser: HttpResponseParser)
extends FlexiMerge[List[ResponseOutput], FanInShape2[ByteString, HttpMethod, List[ResponseOutput]]](new FanInShape2("ResponseParsingMerge"), OperationAttributes.name("ResponsePersingMerge")) {
import FlexiMerge._
val dataInput = createInputPort[ByteString]()
val methodBypassInput = createInputPort[HttpMethod]()
def createMergeLogic() = new MergeLogic[List[ResponseOutput]] {
def createMergeLogic(p: PortT) = new MergeLogic[List[ResponseOutput]] {
val dataInput = p.in0
val methodBypassInput = p.in1
// 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()
var methodBypassCompleted = false
override def inputHandles(inputCount: Int) = {
require(inputCount == 2, s"ResponseParsingMerge must have 2 connected inputs, was $inputCount")
Vector(dataInput, methodBypassInput)
}
private val stay = (ctx: MergeLogicContext) SameState
private val gotoResponseReading = (ctx: MergeLogicContext) {
ctx.changeCompletionHandling(responseReadingCompletionHandling)

View file

@ -254,7 +254,7 @@ private[http] object BodyPartParser {
val boundaryCharNoSpace = CharPredicate.Digit ++ CharPredicate.Alpha ++ "'()+_,-./:=?"
sealed trait Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output] BodyPartEntity) extends Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Source[Output, Unit] BodyPartEntity) extends Output
final case class EntityPart(data: ByteString) extends Output
final case class ParseError(info: ErrorInfo) extends Output

View file

@ -305,7 +305,7 @@ private[http] abstract class HttpMessageParser[Output >: MessageOutput <: Parser
def defaultEntity(cth: Option[`Content-Type`],
contentLength: Long,
transformData: Source[ByteString] Source[ByteString] = identityFunc)(entityParts: Source[_ <: ParserOutput]): UniversalEntity = {
transformData: Source[ByteString, Unit] Source[ByteString, Unit] = identityFunc)(entityParts: Source[_ <: ParserOutput, Unit]): UniversalEntity = {
val data = entityParts.collect {
case EntityPart(bytes) bytes
case EntityStreamError(info) throw EntityStreamException(info)
@ -314,7 +314,7 @@ private[http] abstract class HttpMessageParser[Output >: MessageOutput <: Parser
}
def chunkedEntity(cth: Option[`Content-Type`],
transformChunks: Source[HttpEntity.ChunkStreamPart] Source[HttpEntity.ChunkStreamPart] = identityFunc)(entityChunks: Source[_ <: ParserOutput]): RequestEntity = {
transformChunks: Source[HttpEntity.ChunkStreamPart, Unit] Source[HttpEntity.ChunkStreamPart, Unit] = identityFunc)(entityChunks: Source[_ <: ParserOutput, Unit]): RequestEntity = {
val chunks = entityChunks.collect {
case EntityChunk(chunk) chunk
case EntityStreamError(info) throw EntityStreamException(info)

View file

@ -118,7 +118,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`],
expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult =
if (hostHeaderPresent || protocol == HttpProtocols.`HTTP/1.0`) {
def emitRequestStart(createEntity: Source[RequestOutput] RequestEntity,
def emitRequestStart(createEntity: Source[RequestOutput, Unit] RequestEntity,
headers: List[HttpHeader] = headers) = {
val allHeaders =
if (rawRequestUriHeader) `Raw-Request-URI`(new String(uriBytes, HttpCharsets.`US-ASCII`.nioCharset)) :: headers
@ -126,7 +126,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
emit(RequestStart(method, uri, protocol, allHeaders, createEntity, expect100continue, closeAfterResponseCompletion))
}
def expect100continueHandling[T]: Source[T] Source[T] =
def expect100continueHandling[T]: Source[T, Unit] Source[T, Unit] =
if (expect100continue) {
_.section(name("expect100continueTrigger"))(_.transform(() new PushPullStage[T, T] {
private var oneHundredContinueSent = false

View file

@ -78,7 +78,7 @@ private[http] class HttpResponseParser(_settings: ParserSettings, _headerParser:
def parseEntity(headers: List[HttpHeader], protocol: HttpProtocol, input: ByteString, bodyStart: Int,
clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`],
expect100continue: Boolean, hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult = {
def emitResponseStart(createEntity: Source[ResponseOutput] ResponseEntity,
def emitResponseStart(createEntity: Source[ResponseOutput, Unit] ResponseEntity,
headers: List[HttpHeader] = headers) =
emit(ResponseStart(statusCode, protocol, headers, createEntity, closeAfterResponseCompletion))
def finishEmptyResponse() = {

View file

@ -28,7 +28,7 @@ private[http] object ParserOutput {
uri: Uri,
protocol: HttpProtocol,
headers: List[HttpHeader],
createEntity: Source[RequestOutput] RequestEntity,
createEntity: Source[RequestOutput, Unit] RequestEntity,
expect100ContinueResponsePending: Boolean,
closeAfterResponseCompletion: Boolean) extends MessageStart with RequestOutput
@ -36,7 +36,7 @@ private[http] object ParserOutput {
statusCode: StatusCode,
protocol: HttpProtocol,
headers: List[HttpHeader],
createEntity: Source[ResponseOutput] ResponseEntity,
createEntity: Source[ResponseOutput, Unit] ResponseEntity,
closeAfterResponseCompletion: Boolean) extends MessageStart with ResponseOutput
case object MessageEnd extends MessageOutput

View file

@ -24,19 +24,19 @@ private[http] object BodyPartRenderer {
def streamed(boundary: String,
nioCharset: Charset,
partHeadersSizeHint: Int,
log: LoggingAdapter): PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] =
new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart]] {
log: LoggingAdapter): PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart, Unit]] =
new PushPullStage[Multipart.BodyPart, Source[ChunkStreamPart, Unit]] {
var firstBoundaryRendered = false
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart]]): Directive = {
override def onPush(bodyPart: Multipart.BodyPart, ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
val r = new CustomCharsetByteStringRendering(nioCharset, partHeadersSizeHint)
def bodyPartChunks(data: Source[ByteString]): Source[ChunkStreamPart] = {
def bodyPartChunks(data: Source[ByteString, Unit]): Source[ChunkStreamPart, Unit] = {
val entityChunks = data.map[ChunkStreamPart](Chunk(_))
chunkStream(r.get) ++ entityChunks
(chunkStream(r.get) ++ entityChunks).mapMaterialized((_) ())
}
def completePartRendering(): Source[ChunkStreamPart] =
def completePartRendering(): Source[ChunkStreamPart, Unit] =
bodyPart.entity match {
case x if x.isKnownEmpty chunkStream(r.get)
case Strict(_, data) chunkStream((r ~~ data).get)
@ -51,7 +51,7 @@ private[http] object BodyPartRenderer {
ctx.push(completePartRendering())
}
override def onPull(ctx: Context[Source[ChunkStreamPart]]): Directive = {
override def onPull(ctx: Context[Source[ChunkStreamPart, Unit]]): Directive = {
val finishing = ctx.isFinishing
if (finishing && firstBoundaryRendered) {
val r = new ByteStringRendering(boundary.length + 4)
@ -63,9 +63,9 @@ private[http] object BodyPartRenderer {
ctx.pull()
}
override def onUpstreamFinish(ctx: Context[Source[ChunkStreamPart]]): TerminationDirective = ctx.absorbTermination()
override def onUpstreamFinish(ctx: Context[Source[ChunkStreamPart, Unit]]): TerminationDirective = ctx.absorbTermination()
private def chunkStream(byteString: ByteString): Source[ChunkStreamPart] =
private def chunkStream(byteString: ByteString): Source[ChunkStreamPart, Unit] =
Source.single(Chunk(byteString))
}

View file

@ -25,9 +25,9 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
def newRenderer: HttpRequestRenderer = new HttpRequestRenderer
final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString]] {
final class HttpRequestRenderer extends PushStage[RequestRenderingContext, Source[ByteString, Unit]] {
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString]]): Directive = {
override def onPush(ctx: RequestRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
val r = new ByteStringRendering(requestHeaderSizeHint)
import ctx.request._
@ -102,7 +102,7 @@ private[http] class HttpRequestRendererFactory(userAgentHeader: Option[headers.`
def renderContentLength(contentLength: Long) =
if (method.isEntityAccepted) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r
def completeRequestRendering(): Source[ByteString] =
def completeRequestRendering(): Source[ByteString, Unit] =
entity match {
case x if x.isKnownEmpty
renderContentLength(0) ~~ CrLf

View file

@ -51,14 +51,14 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
def newRenderer: HttpResponseRenderer = new HttpResponseRenderer
final class HttpResponseRenderer extends PushStage[ResponseRenderingContext, Source[ByteString]] {
final class HttpResponseRenderer extends PushStage[ResponseRenderingContext, Source[ByteString, Unit]] {
private[this] var close = false // signals whether the connection is to be closed after the current response
// need this for testing
private[http] def isComplete = close
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString]]): Directive = {
override def onPush(ctx: ResponseRenderingContext, opCtx: Context[Source[ByteString, Unit]]): Directive = {
val r = new ByteStringRendering(responseHeaderSizeHint)
import ctx.response._
@ -140,10 +140,10 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
def renderContentLengthHeader(contentLength: Long) =
if (status.allowsEntity) r ~~ `Content-Length` ~~ contentLength ~~ CrLf else r
def byteStrings(entityBytes: Source[ByteString]): Source[ByteString] =
def byteStrings(entityBytes: Source[ByteString, Unit]): Source[ByteString, Unit] =
renderByteStrings(r, entityBytes, skipEntity = noEntity)
def completeResponseRendering(entity: ResponseEntity): Source[ByteString] =
def completeResponseRendering(entity: ResponseEntity): Source[ByteString, Unit] =
entity match {
case HttpEntity.Strict(_, data)
renderHeaders(headers.toList)

View file

@ -5,7 +5,6 @@
package akka.http.engine.rendering
import akka.parboiled2.CharUtils
import akka.stream.ActorFlowMaterializer
import akka.util.ByteString
import akka.event.LoggingAdapter
import akka.stream.scaladsl._
@ -30,25 +29,24 @@ private object RenderSupport {
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
// This hooks into the materialization to cancel the not needed second source. This helper class
// allows us to not take a FlowMaterializer but delegate the cancellation to the point when the whole stream
// materializes
private case class CancelSecond[T](first: Source[T], second: Source[T]) extends SimpleActorFlowSource[T] {
override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit = {
first.to(Sink(flowSubscriber)).run()(materializer)
second.to(Sink.cancelled).run()(materializer)
}
def CancelSecond[T](first: Source[T, _], second: Source[T, _]): Source[T, Unit] = {
Source(first) { implicit b
frst
import FlowGraph.Implicits._
second ~> Sink.cancelled
frst.outlet
}.mapMaterialized((_) ())
}
def renderEntityContentType(r: Rendering, entity: HttpEntity) =
if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf
else r
def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString],
skipEntity: Boolean = false): Source[ByteString] = {
def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString, Unit],
skipEntity: Boolean = false): Source[ByteString, Unit] = {
val messageStart = Source.single(r.get)
val messageBytes =
if (!skipEntity) messageStart ++ entityBytes
if (!skipEntity) (messageStart ++ entityBytes).mapMaterialized((_) ())
else CancelSecond(messageStart, entityBytes)
messageBytes
}

View file

@ -4,13 +4,15 @@
package akka.http.engine.server
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.scaladsl._
import akka.stream._
import scala.collection.immutable
import scala.util.control.NonFatal
import akka.actor.{ ActorRef, Props }
import akka.util.ByteString
import akka.event.LoggingAdapter
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlattenStrategy
import akka.stream.scaladsl._
import akka.stream.stage.PushPullStage
import akka.http.engine.parsing.{ HttpHeaderParser, HttpRequestParser }
import akka.http.engine.rendering.{ ResponseRenderingContext, HttpResponseRendererFactory }
@ -18,17 +20,37 @@ import akka.http.engine.parsing.ParserOutput._
import akka.http.engine.TokenSourceActor
import akka.http.model._
import akka.http.util._
import akka.stream.FlowMaterializer
import akka.stream.OverflowStrategy
/**
* INTERNAL API
*/
private[http] object HttpServer {
def serverFlowToTransport(serverFlow: Flow[HttpRequest, HttpResponse],
settings: ServerSettings,
log: LoggingAdapter)(implicit mat: FlowMaterializer): Flow[ByteString, ByteString] = {
case class HttpServerPorts(
bytesIn: Inlet[ByteString],
bytesOut: Outlet[ByteString],
httpResponses: Inlet[HttpResponse],
httpRequests: Outlet[HttpRequest]) extends Shape {
override def inlets: immutable.Seq[Inlet[_]] = bytesIn :: httpResponses :: Nil
override def outlets: immutable.Seq[Outlet[_]] = bytesOut :: httpRequests :: Nil
override def deepCopy() = HttpServerPorts(
new Inlet(bytesIn.toString),
new Outlet(bytesOut.toString),
new Inlet(httpRequests.toString),
new Outlet(httpResponses.toString))
override def copyFromPorts(inlets: immutable.Seq[Inlet[_]], outlets: immutable.Seq[Outlet[_]]): Shape = {
require(inlets.size == 2, s"proposed inlets [${inlets.mkString(", ")}] do not fit BidiShape")
require(outlets.size == 2, s"proposed outlets [${outlets.mkString(", ")}] do not fit BidiShape")
HttpServerPorts(inlets(0).asInstanceOf[Inlet[ByteString]], outlets(0).asInstanceOf[Outlet[ByteString]],
inlets(1).asInstanceOf[Inlet[HttpResponse]], outlets(1).asInstanceOf[Outlet[HttpRequest]])
}
}
def serverBlueprint(settings: ServerSettings,
log: LoggingAdapter)(implicit mat: ActorFlowMaterializer): Graph[HttpServerPorts, Unit] = {
// the initial header parser we initially use for every connection,
// will not be mutated, all "shared copy" parsers copy on first-write into the header cache
@ -50,10 +72,7 @@ private[http] object HttpServer {
}
}
val bypassFanout = Broadcast[RequestOutput](OperationAttributes.name("bypassFanout"))
val bypassMerge = new BypassMerge(settings, log)
val requestParsing = Flow[ByteString].section(name("rootParser"))(_.transform(()
val requestParsingFlow = Flow[ByteString].section(name("rootParser"))(_.transform(()
// each connection uses a single (private) request parser instance for all its requests
// which builds a cache of all header instances seen on that connection
rootParser.createShallowCopy(() oneHundredContinueRef).stage))
@ -67,7 +86,7 @@ private[http] object HttpServer {
val effectiveUri = HttpRequest.effectiveUri(uri, headers, securedConnection = false, settings.defaultHostHeader)
val effectiveMethod = if (method == HttpMethods.HEAD && settings.transparentHeadRequests) HttpMethods.GET else method
HttpRequest(effectiveMethod, effectiveUri, headers, createEntity(entityParts), protocol)
case (_, src) src.runWith(BlackholeSink)
case (_, src) src.runWith(Sink.ignore)
}.collect {
case r: HttpRequest r
}.buffer(1, OverflowStrategy.backpressure)
@ -89,39 +108,44 @@ private[http] object HttpServer {
.flatten(FlattenStrategy.concat)
.section(name("errorLogger"))(_.transform(() errorLogger(log, "Outgoing response stream error")))
val transportIn = UndefinedSource[ByteString]
val transportOut = UndefinedSink[ByteString]
FlowGraph.partial(requestParsingFlow, rendererPipeline)(Keep.right) { implicit b
(requestParsing, renderer)
import FlowGraph.Implicits._
import FlowGraphImplicits._
val bypassFanout = b.add(Broadcast[RequestOutput](2, OperationAttributes.name("bypassFanout")))
val bypassMerge = b.add(new BypassMerge(settings, log))
val bypassInput = bypassMerge.in0
val bypassOneHundredContinueInput = bypassMerge.in1
val bypassApplicationInput = bypassMerge.in2
Flow() { implicit b
//FIXME: the graph is unnecessary after fixing #15957
transportIn ~> requestParsing ~> bypassFanout ~> requestPreparation ~> serverFlow ~> bypassMerge.applicationInput ~> rendererPipeline ~> transportOut
bypassFanout ~> bypass ~> bypassMerge.bypassInput
oneHundredContinueSource ~> bypassMerge.oneHundredContinueInput
requestParsing.outlet ~> bypassFanout.in
bypassMerge.out ~> renderer.inlet
val requestsIn = (bypassFanout.out(0) ~> requestPreparation).outlet
b.allowCycles()
bypassFanout.out(1) ~> bypass ~> bypassInput
oneHundredContinueSource ~> bypassOneHundredContinueInput
transportIn -> transportOut
HttpServerPorts(
requestParsing.inlet,
renderer.outlet,
bypassApplicationInput,
requestsIn)
}
}
class BypassMerge(settings: ServerSettings, log: LoggingAdapter)
extends FlexiMerge[ResponseRenderingContext](OperationAttributes.name("BypassMerge")) {
extends FlexiMerge[ResponseRenderingContext, FanInShape3[RequestOutput, OneHundredContinue.type, HttpResponse, ResponseRenderingContext]](new FanInShape3("BypassMerge"), OperationAttributes.name("BypassMerge")) {
import FlexiMerge._
val bypassInput = createInputPort[RequestOutput]()
val oneHundredContinueInput = createInputPort[OneHundredContinue.type]()
val applicationInput = createInputPort[HttpResponse]()
def createMergeLogic() = new MergeLogic[ResponseRenderingContext] {
def createMergeLogic(p: PortT) = new MergeLogic[ResponseRenderingContext] {
var requestStart: RequestStart = _
override def inputHandles(inputCount: Int) = {
require(inputCount == 3, s"BypassMerge must have 3 connected inputs, was $inputCount")
Vector(bypassInput, oneHundredContinueInput, applicationInput)
}
val bypassInput: Inlet[RequestOutput] = p.in0
val oneHundredContinueInput: Inlet[OneHundredContinue.type] = p.in1
val applicationInput: Inlet[HttpResponse] = p.in2
override val initialState: State[Any] = State[Any](Read(bypassInput)) {
override val initialState: State[RequestOutput] = State[RequestOutput](Read(bypassInput)) {
case (ctx, _, requestStart: RequestStart)
this.requestStart = requestStart
ctx.changeCompletionHandling(waitingForApplicationResponseCompletionHandling)
@ -133,7 +157,7 @@ private[http] object HttpServer {
override val initialCompletionHandling = eagerClose
val waitingForApplicationResponse =
State[Any](ReadAny(oneHundredContinueInput, applicationInput)) {
State[Any](ReadAny(oneHundredContinueInput.asInstanceOf[Inlet[Any]] :: applicationInput.asInstanceOf[Inlet[Any]] :: Nil)) {
case (ctx, _, response: HttpResponse)
// see the comment on [[OneHundredContinue]] for an explanation of the closing logic here (and more)
val close = requestStart.closeAfterResponseCompletion || requestStart.expect100ContinueResponsePending

View file

@ -13,7 +13,7 @@ import scala.collection.immutable
import scala.util.control.NonFatal
import akka.util.ByteString
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl._
import akka.stream.TimerTransformer
import akka.http.util._
@ -38,13 +38,13 @@ sealed trait HttpEntity extends japi.HttpEntity {
/**
* A stream of the data of this entity.
*/
def dataBytes: Source[ByteString]
def dataBytes: Source[ByteString, Unit]
/**
* Collects all possible parts and returns a potentially future Strict entity for easier processing.
* The Future is failed with an TimeoutException if the stream isn't completed after the given timeout.
*/
def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer): Future[HttpEntity.Strict] = {
def toStrict(timeout: FiniteDuration)(implicit fm: ActorFlowMaterializer): Future[HttpEntity.Strict] = {
def transformer() =
new TimerTransformer[ByteString, HttpEntity.Strict] {
var bytes = ByteString.newBuilder
@ -64,7 +64,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
}
// TODO timerTransform is meant to be replaced / rewritten, it's currently private[akka]; See https://github.com/akka/akka/issues/16393
dataBytes.section(name("toStrict"))(_.timerTransform(transformer)).runWith(Sink.head)
dataBytes.section(name("toStrict"))(_.timerTransform(transformer)).runWith(Sink.head())
}
/**
@ -75,7 +75,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: Flow[ByteString, ByteString]): HttpEntity
def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): HttpEntity
/**
* Creates a copy of this HttpEntity with the `contentType` overridden with the given one.
@ -83,7 +83,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
def withContentType(contentType: ContentType): HttpEntity
/** Java API */
def getDataBytes: Source[ByteString] = dataBytes
def getDataBytes: Source[ByteString, Unit] = dataBytes
// default implementations, should be overridden
def isCloseDelimited: Boolean = false
@ -100,13 +100,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: Flow[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: Flow[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 {
@ -117,7 +117,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: Flow[ByteString, ByteString]): UniversalEntity
def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity
}
object HttpEntity {
@ -130,7 +130,7 @@ object HttpEntity {
if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes))
def apply(contentType: ContentType, data: ByteString): Strict =
if (data.isEmpty) empty(contentType) else Strict(contentType, data)
def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString]): UniversalEntity =
def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString, Unit]): UniversalEntity =
if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data)
def apply(contentType: ContentType, file: File): UniversalEntity = {
@ -148,50 +148,26 @@ 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
def dataBytes: Source[ByteString] = Source(data :: Nil)
def dataBytes: Source[ByteString, Unit] = Source(data :: Nil)
override def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer) =
override def toStrict(timeout: FiniteDuration)(implicit fm: ActorFlowMaterializer) =
FastFuture.successful(this)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): MessageEntity =
StreamUtils.runStrict(data, transformer, MaxByteSize, MaxElements) match {
case Success(Some(newData))
copy(data = newData)
case Success(None)
Chunked.fromData(contentType, Source.single(data).via(transformer))
case Failure(ex)
Chunked(contentType, Source.failed(ex))
}
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): MessageEntity =
Chunked.fromData(contentType, Source.single(data).via(transformer))
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.single(data).via(transformer))
case Failure(ex)
Default(contentType, newContentLength, Source.failed(ex))
}
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity =
Default(contentType, newContentLength, Source.single(data).via(transformer))
def withContentType(contentType: ContentType): Strict =
if (contentType == this.contentType) this else copy(contentType = contentType)
@ -204,20 +180,20 @@ object HttpEntity {
*/
final case class Default(contentType: ContentType,
contentLength: Long,
data: Source[ByteString])
data: Source[ByteString, Unit])
extends japi.HttpEntityDefault with UniversalEntity {
require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)")
def isKnownEmpty = false
override def isDefault: Boolean = true
def dataBytes: Source[ByteString] = data
def dataBytes: Source[ByteString, Unit] = data
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked =
Chunked.fromData(contentType, data.via(transformer))
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): Chunked =
Chunked.fromData(contentType, data.viaMat(transformer)(Keep.left))
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString]): UniversalEntity =
Default(contentType, newContentLength, data.via(transformer))
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity =
Default(contentType, newContentLength, data.viaMat(transformer)(Keep.left))
def withContentType(contentType: ContentType): Default =
if (contentType == this.contentType) this else copy(contentType = contentType)
@ -232,11 +208,11 @@ object HttpEntity {
*/
private[http] sealed trait WithoutKnownLength extends HttpEntity {
def contentType: ContentType
def data: Source[ByteString]
def data: Source[ByteString, Unit]
def isKnownEmpty = data eq Source.empty
def dataBytes: Source[ByteString] = data
def dataBytes: Source[ByteString, Unit] = data
}
/**
@ -244,7 +220,7 @@ object HttpEntity {
* The content-length of such responses is unknown at the time the response headers have been received.
* Note that this type of HttpEntity can only be used for HttpResponses.
*/
final case class CloseDelimited(contentType: ContentType, data: Source[ByteString])
final case class CloseDelimited(contentType: ContentType, data: Source[ByteString, Unit])
extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength {
type Self = CloseDelimited
@ -252,8 +228,8 @@ object HttpEntity {
def withContentType(contentType: ContentType): CloseDelimited =
if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): CloseDelimited =
HttpEntity.CloseDelimited(contentType, data.via(transformer))
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): CloseDelimited =
HttpEntity.CloseDelimited(contentType, data.viaMat(transformer)(Keep.left))
override def productPrefix = "HttpEntity.CloseDelimited"
}
@ -262,15 +238,15 @@ object HttpEntity {
* The model for the entity of a BodyPart with an indefinite length.
* Note that this type of HttpEntity can only be used for BodyParts.
*/
final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString])
final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString, Unit])
extends japi.HttpEntityIndefiniteLength with BodyPartEntity with WithoutKnownLength {
override def isIndefiniteLength: Boolean = true
def withContentType(contentType: ContentType): IndefiniteLength =
if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): IndefiniteLength =
HttpEntity.IndefiniteLength(contentType, data.via(transformer))
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): IndefiniteLength =
HttpEntity.IndefiniteLength(contentType, data.viaMat(transformer)(Keep.left))
override def productPrefix = "HttpEntity.IndefiniteLength"
}
@ -278,23 +254,23 @@ object HttpEntity {
/**
* The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`).
*/
final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart])
final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart, Unit])
extends japi.HttpEntityChunked with MessageEntity {
def isKnownEmpty = chunks eq Source.empty
override def isChunked: Boolean = true
def dataBytes: Source[ByteString] =
def dataBytes: Source[ByteString, Unit] =
chunks.map(_.data).filter(_.nonEmpty)
override def transformDataBytes(transformer: Flow[ByteString, ByteString]): Chunked = {
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)
}.viaMat(transformer)(Keep.left)
Chunked.fromData(contentType, newData)
}
@ -305,14 +281,14 @@ object HttpEntity {
override def productPrefix = "HttpEntity.Chunked"
/** Java API */
def getChunks: Source[japi.ChunkStreamPart] = chunks.asInstanceOf[Source[japi.ChunkStreamPart]]
def getChunks: Source[japi.ChunkStreamPart, Unit] = chunks.asInstanceOf[Source[japi.ChunkStreamPart, Unit]]
}
object Chunked {
/**
* Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given
* ``Publisher[ByteString]``.
*/
def fromData(contentType: ContentType, chunks: Source[ByteString]): Chunked =
def fromData(contentType: ContentType, chunks: Source[ByteString, Unit]): Chunked =
Chunked(contentType, chunks.collect[ChunkStreamPart] {
case b: ByteString if b.nonEmpty Chunk(b)
})

View file

@ -11,7 +11,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable
import scala.reflect.{ classTag, ClassTag }
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
import akka.util.ByteString
import akka.http.util._
import headers._
@ -51,7 +51,7 @@ sealed trait HttpMessage extends japi.HttpMessage {
def withEntity(entity: MessageEntity): Self
/** Returns a sharable and serializable copy of this message with a strict entity. */
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Self] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Self] =
entity.toStrict(timeout).fast.map(this.withEntity)
/** Returns a copy of this message with the entity and headers set to the given ones. */

View file

@ -9,7 +9,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{ Future, ExecutionContext }
import scala.collection.immutable
import scala.util.{ Failure, Success, Try }
import akka.stream.FlowMaterializer
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.Source
import akka.http.util.FastFuture
import akka.http.model.headers._
@ -17,14 +17,14 @@ import FastFuture._
trait Multipart {
def mediaType: MultipartMediaType
def parts: Source[Multipart.BodyPart]
def parts: Source[Multipart.BodyPart, Unit]
/**
* Converts this content into its strict counterpart.
* The given ``timeout`` denotes the max time that an individual part must be read in.
* The Future is failed with an TimeoutException if one part isn't read completely after the given timeout.
*/
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Multipart.Strict]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Multipart.Strict]
}
object Multipart {
@ -47,7 +47,7 @@ object Multipart {
def dispositionType: Option[ContentDispositionType] =
contentDispositionHeader.map(_.dispositionType)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict]
}
object BodyPart {
@ -56,7 +56,7 @@ object Multipart {
}
}
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Vector[BPS]] =
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Unit])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Vector[BPS]] =
// TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed
parts.runFold(new VectorBuilder[Future[BPS]]) {
case (builder, part) builder += f(part)
@ -69,28 +69,28 @@ object Multipart {
*/
sealed abstract class General extends Multipart {
def mediaType: MultipartMediaType
def parts: Source[General.BodyPart]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[General.Strict] =
def parts: Source[General.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[General.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _))
}
object General {
def apply(mediaType: MultipartMediaType, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector)
def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart]): General =
def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart, Unit]): General =
new General {
def mediaType = _mediaType
def parts = _parts
override def toString = s"General($mediaType, $parts)"
}
def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart])] = Some(value.mediaType -> value.parts)
def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart, Unit])] = Some(value.mediaType -> value.parts)
/**
* Strict [[General]].
*/
case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this)
override def productPrefix = "General.Strict"
}
@ -99,7 +99,7 @@ object Multipart {
* Body part of the [[General]] model.
*/
sealed abstract class BodyPart extends Multipart.BodyPart {
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(_, headers))
def toFormDataBodyPart: Try[FormData.BodyPart]
def toByteRangesBodyPart: Try[ByteRanges.BodyPart]
@ -133,7 +133,7 @@ object Multipart {
* Strict [[General.BodyPart]].
*/
case class Strict(entity: HttpEntity.Strict, headers: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this)
override def toFormDataBodyPart: Try[FormData.BodyPart.Strict] = tryCreateFormDataBodyPart(FormData.BodyPart.Strict(_, entity, _, _))
override def toByteRangesBodyPart: Try[ByteRanges.BodyPart.Strict] = tryCreateByteRangesBodyPart(ByteRanges.BodyPart.Strict(_, entity, _, _))
@ -148,8 +148,8 @@ object Multipart {
*/
sealed abstract class FormData extends Multipart {
def mediaType = MediaTypes.`multipart/form-data`
def parts: Source[FormData.BodyPart]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[FormData.Strict] =
def parts: Source[FormData.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[FormData.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_))
}
object FormData {
@ -159,7 +159,7 @@ object Multipart {
fields.map { case (name, entity) BodyPart.Strict(name, entity) }(collection.breakOut)
}
def apply(_parts: Source[BodyPart]): FormData = new FormData {
def apply(_parts: Source[BodyPart, Unit]): FormData = new FormData {
def parts = _parts
override def toString = s"FormData($parts)"
}
@ -168,8 +168,8 @@ object Multipart {
* Strict [[FormData]].
*/
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this)
override def productPrefix = "FormData.Strict"
}
@ -186,7 +186,7 @@ object Multipart {
override def dispositionParams = additionalDispositionParams.updated("name", name)
override def dispositionType = Some(ContentDispositionTypes.`form-data`)
def filename: Option[String] = additionalDispositionParams.get("filename")
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(name, _, additionalDispositionParams, additionalHeaders))
}
object BodyPart {
@ -210,7 +210,7 @@ object Multipart {
case class Strict(name: String, entity: HttpEntity.Strict,
additionalDispositionParams: Map[String, String] = Map.empty,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this)
override def productPrefix = "FormData.BodyPart.Strict"
}
@ -223,14 +223,14 @@ object Multipart {
*/
sealed abstract class ByteRanges extends Multipart {
def mediaType = MediaTypes.`multipart/byteranges`
def parts: Source[ByteRanges.BodyPart]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[ByteRanges.Strict] =
def parts: Source[ByteRanges.BodyPart, Unit]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[ByteRanges.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_))
}
object ByteRanges {
def apply(parts: BodyPart.Strict*): Strict = Strict(parts.toVector)
def apply(_parts: Source[BodyPart]): ByteRanges =
def apply(_parts: Source[BodyPart, Unit]): ByteRanges =
new ByteRanges {
def parts = _parts
override def toString = s"ByteRanges($parts)"
@ -240,8 +240,8 @@ object Multipart {
* Strict [[ByteRanges]].
*/
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict {
def parts: Source[BodyPart.Strict] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer) =
FastFuture.successful(this)
override def productPrefix = "ByteRanges.Strict"
}
@ -255,7 +255,7 @@ object Multipart {
def additionalHeaders: immutable.Seq[HttpHeader]
override def headers = contentRangeHeader +: additionalHeaders
def contentRangeHeader = `Content-Range`(rangeUnit, contentRange)
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[BodyPart.Strict] =
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[BodyPart.Strict] =
entity.toStrict(timeout).map(BodyPart.Strict(contentRange, _, rangeUnit, additionalHeaders))
}
object BodyPart {
@ -277,7 +277,7 @@ object Multipart {
*/
case class Strict(contentRange: ContentRange, entity: HttpEntity.Strict, rangeUnit: RangeUnit = RangeUnits.Bytes,
additionalHeaders: immutable.Seq[HttpHeader] = Nil) extends BodyPart with Multipart.BodyPart.Strict {
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Strict] =
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: ActorFlowMaterializer): Future[Strict] =
FastFuture.successful(this)
override def productPrefix = "ByteRanges.BodyPart.Strict"
}

View file

@ -14,15 +14,11 @@ import scala.concurrent.{ ExecutionContext, Future }
import scala.util.Try
import akka.actor.Props
import akka.http.model.RequestEntity
import akka.stream.ActorFlowMaterializer
import akka.stream.FlowMaterializer
import akka.stream.impl.Ast.AstNode
import akka.stream.impl.Ast.StageFactory
import akka.stream.{ ActorFlowMaterializerSettings, ActorFlowMaterializer, impl }
import akka.stream.impl.fusing.IteratorInterpreter
import akka.stream.scaladsl._
import akka.stream.scaladsl.OperationAttributes._
import akka.stream.stage._
import akka.stream.impl
import akka.util.ByteString
import org.reactivestreams.{ Subscriber, Publisher }
@ -51,7 +47,7 @@ private[http] object StreamUtils {
def failedPublisher[T](ex: Throwable): Publisher[T] =
impl.ErrorPublisher(ex, "failed").asInstanceOf[Publisher[T]]
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString] = {
def mapErrorTransformer(f: Throwable Throwable): Flow[ByteString, ByteString, Unit] = {
val transformer = new PushStage[ByteString, ByteString] {
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
ctx.push(element)
@ -63,11 +59,12 @@ private[http] object StreamUtils {
Flow[ByteString].section(name("transformError"))(_.transform(() transformer))
}
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString] = {
def sliceBytesTransformer(start: Long, length: Long): Flow[ByteString, ByteString, Unit] = {
val transformer = new StatefulStage[ByteString, ByteString] {
def skipping = new State {
var toSkip = start
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive =
if (element.length < toSkip) {
// keep skipping
@ -79,8 +76,10 @@ private[http] object StreamUtils {
current.onPush(element.drop(toSkip.toInt), ctx)
}
}
def taking(initiallyRemaining: Long) = new State {
var remaining: Long = initiallyRemaining
override def onPush(element: ByteString, ctx: Context[ByteString]): Directive = {
val data = element.take(math.min(remaining, Int.MaxValue).toInt)
remaining -= data.size
@ -94,9 +93,10 @@ private[http] object StreamUtils {
Flow[ByteString].section(name("sliceBytes"))(_.transform(() transformer))
}
def limitByteChunksStage(maxBytesPerChunk: Int): Stage[ByteString, ByteString] =
def limitByteChunksStage(maxBytesPerChunk: Int): PushPullStage[ByteString, ByteString] =
new StatefulStage[ByteString, ByteString] {
def initial = WaitingForData
case object WaitingForData extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
if (elem.size <= maxBytesPerChunk) ctx.push(elem)
@ -105,6 +105,7 @@ private[http] object StreamUtils {
ctx.push(elem.take(maxBytesPerChunk))
}
}
case class DeliveringData(remaining: ByteString) extends State {
def onPush(elem: ByteString, ctx: Context[ByteString]): Directive =
throw new IllegalStateException("Not expecting data")
@ -133,23 +134,21 @@ private[http] object StreamUtils {
* Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source
* will only be traversed once.
*/
def transformMultiple(input: Source[ByteString], transformers: immutable.Seq[Flow[ByteString, ByteString]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString]] =
def transformMultiple(input: Source[ByteString, Unit], transformers: immutable.Seq[Flow[ByteString, ByteString, _]])(implicit materializer: ActorFlowMaterializer): immutable.Seq[Source[ByteString, Unit]] =
transformers match {
case Nil Nil
case Seq(one) Vector(input.via(one))
case multiple
val results = Vector.fill(multiple.size)(Sink.publisher[ByteString])
val mat =
FlowGraph { implicit b
import FlowGraphImplicits._
val (fanoutSub, fanoutPub) = Source.subscriber[ByteString]().toMat(Sink.fanoutPublisher(16, 16))(Keep.both).run()
val sources = transformers.map { flow
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources
// are exposed, so there is no race
Source(Source(fanoutPub).via(flow).runWith(Sink.publisher()))
}
// The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
input.runWith(Sink(fanoutSub))
sources
val broadcast = Broadcast[ByteString](OperationAttributes.name("transformMultipleInputBroadcast"))
input ~> broadcast
(multiple, results).zipped.foreach { (trans, sink)
broadcast ~> trans ~> sink
}
}.run()
results.map(s Source(mat.get(s)))
}
def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity =
@ -160,118 +159,55 @@ private[http] object StreamUtils {
*
* FIXME: should be provided by akka-stream, see #15588
*/
def fromInputStreamSource(inputStream: InputStream, defaultChunkSize: Int = 65536): Source[ByteString] = {
def fromInputStreamSource(inputStream: InputStream,
fileIODispatcher: String,
defaultChunkSize: Int = 65536): Source[ByteString, Unit] = {
import akka.stream.impl._
def props(materializer: ActorFlowMaterializer): Props = {
val iterator = new Iterator[ByteString] {
var finished = false
def hasNext: Boolean = !finished
def next(): ByteString =
if (!finished) {
val buffer = new Array[Byte](defaultChunkSize)
val read = inputStream.read(buffer)
if (read < 0) {
finished = true
inputStream.close()
ByteString.empty
} else ByteString.fromArray(buffer, 0, read)
} else ByteString.empty
}
val onlyOnceFlag = new AtomicBoolean(false)
IteratorPublisher.props(iterator, materializer.settings).withDispatcher(materializer.settings.fileIODispatcher)
val iterator = new Iterator[ByteString] {
var finished = false
if (onlyOnceFlag.get() || !onlyOnceFlag.compareAndSet(false, true))
throw new IllegalStateException("One time source can only be instantiated once")
def hasNext: Boolean = !finished
def next(): ByteString =
if (!finished) {
val buffer = new Array[Byte](defaultChunkSize)
val read = inputStream.read(buffer)
if (read < 0) {
finished = true
inputStream.close()
ByteString.empty
} else ByteString.fromArray(buffer, 0, read)
} else ByteString.empty
}
new AtomicBoolean(false) with SimpleActorFlowSource[ByteString] {
override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber)
Source(() iterator).withAttributes(OperationAttributes.dispatcher(fileIODispatcher))
override def isActive: Boolean = true
override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[ByteString], Unit) =
if (!getAndSet(true)) {
val ref = materializer.actorOf(props(materializer), name = s"$flowName-0-InputStream-source")
val publisher = ActorPublisher[ByteString](ref)
ref ! ExposedPublisher(publisher.asInstanceOf[impl.ActorPublisher[Any]])
(publisher, ())
} else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[ByteString]], ())
}
}
/**
* Returns a source that can only be used once for testing purposes.
*/
def oneTimeSource[T](other: Source[T]): Source[T] = {
import akka.stream.impl._
val original = other.asInstanceOf[ActorFlowSource[T]]
new AtomicBoolean(false) with SimpleActorFlowSource[T] {
override def attach(flowSubscriber: Subscriber[T], materializer: ActorFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber)
override def isActive: Boolean = true
override def create(materializer: ActorFlowMaterializer, flowName: String): (Publisher[T], Unit) =
if (!getAndSet(true)) (original.create(materializer, flowName)._1, ())
else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once"), "failed").asInstanceOf[Publisher[T]], ())
def oneTimeSource[T, Mat](other: Source[T, Mat]): Source[T, Mat] = {
val onlyOnceFlag = new AtomicBoolean(false)
other.map { elem
if (onlyOnceFlag.get() || !onlyOnceFlag.compareAndSet(false, true))
throw new IllegalStateException("One time source can only be instantiated once")
elem
}
}
def runStrict(sourceData: ByteString, transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] =
runStrict(Iterator.single(sourceData), transformer, maxByteSize, maxElements)
def runStrict(sourceData: Iterator[ByteString], transformer: Flow[ByteString, ByteString], maxByteSize: Long, maxElements: Int): Try[Option[ByteString]] =
Try {
transformer match {
// FIXME #16382 right now the flow can't use keys, should that be allowed?
case Pipe(ops, keys, _) if keys.isEmpty
if (ops.isEmpty)
Some(sourceData.foldLeft(ByteString.empty)(_ ++ _))
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(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 EnhancedByteStringSource(val byteStringStream: Source[ByteString]) extends AnyVal {
def join(implicit materializer: FlowMaterializer): Future[ByteString] =
private[http] class EnhancedByteStringSource[Mat](val byteStringStream: Source[ByteString, Mat]) extends AnyVal {
def join(implicit materializer: ActorFlowMaterializer): Future[ByteString] =
byteStringStream.runFold(ByteString.empty)(_ ++ _)
def utf8String(implicit materializer: FlowMaterializer, ec: ExecutionContext): Future[String] =
def utf8String(implicit materializer: ActorFlowMaterializer, ec: ExecutionContext): Future[String] =
join.map(_.utf8String)
}

View file

@ -9,7 +9,7 @@ import language.higherKinds
import scala.collection.immutable
import java.nio.charset.Charset
import com.typesafe.config.Config
import akka.stream.{ FlowMaterializer, FlattenStrategy }
import akka.stream.{ ActorFlowMaterializer, FlattenStrategy }
import akka.stream.scaladsl.{ Flow, Source }
import akka.stream.stage._
import scala.concurrent.duration.Duration
@ -40,22 +40,22 @@ package object util {
private[http] implicit def enhanceRegex(regex: Regex): EnhancedRegex = new EnhancedRegex(regex)
private[http] implicit def enhanceByteStrings(byteStrings: TraversableOnce[ByteString]): EnhancedByteStringTraversableOnce =
new EnhancedByteStringTraversableOnce(byteStrings)
private[http] implicit def enhanceByteStrings(byteStrings: Source[ByteString]): EnhancedByteStringSource =
private[http] implicit def enhanceByteStrings[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] =
new EnhancedByteStringSource(byteStrings)
private[http] implicit class SourceWithHeadAndTail[T](val underlying: Source[Source[T]]) extends AnyVal {
def headAndTail: Source[(T, Source[T])] =
private[http] implicit class SourceWithHeadAndTail[T, Mat](val underlying: Source[Source[T, Unit], Mat]) extends AnyVal {
def headAndTail: Source[(T, Source[T, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat)
}
private[http] implicit class FlowWithHeadAndTail[In, Out](val underlying: Flow[In, Source[Out]]) extends AnyVal {
def headAndTail: Flow[In, (Out, Source[Out])] =
private[http] implicit class FlowWithHeadAndTail[In, Out, Mat](val underlying: Flow[In, Source[Out, Unit], Mat]) extends AnyVal {
def headAndTail: Flow[In, (Out, Source[Out, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat)
}
def printEvent[T](marker: String): Flow[T, T] =
def printEvent[T](marker: String): Flow[T, T, Unit] =
Flow[T].transform(() new PushStage[T, T] {
override def onPush(element: T, ctx: Context[T]): Directive = {
println(s"$marker: $element")

View file

@ -6,18 +6,6 @@ package akka.http.model.japi;
import static akka.pattern.Patterns.ask;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.dispatch.Foreach;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.Source;
import akka.stream.FlowMaterializer;
import scala.concurrent.Future;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
public abstract class JavaTestServer {
// FIXME Java Http API

View file

@ -6,6 +6,7 @@ package akka.http
import java.io.{ BufferedReader, BufferedWriter, InputStreamReader, OutputStreamWriter }
import java.net.Socket
import akka.stream.impl.{ PublisherSink, SubscriberSource }
import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.tailrec
import scala.concurrent.Await
@ -39,12 +40,10 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
"properly bind a server" in {
val (hostname, port) = temporaryServerHostnameAndPort()
val binding = Http().bind(hostname, port)
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm = binding.connections.to(Sink(probe)).run()
val sub = probe.expectSubscription()
// if the future finishes successfully, we are bound
val address = Await.result(binding.localAddress(mm), 1.second)
val binding = Http().bind(hostname, port).toMat(Sink(probe))(Keep.left).run()
val sub = probe.expectSubscription() // if we get it we are bound
val address = Await.result(binding, 1.second).localAddress
sub.cancel()
}
@ -52,40 +51,30 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val (hostname, port) = temporaryServerHostnameAndPort()
val binding = Http().bind(hostname, port)
val probe1 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm1 = binding.connections.to(Sink(probe1)).run()
// Bind succeeded, we have a local address
val b1 = Await.result(binding.to(Sink(probe1)).run(), 3.seconds)
probe1.expectSubscription()
// Bind succeeded, we have a local address
Await.result(binding.localAddress(mm1), 1.second)
val probe2 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm2 = binding.connections.to(Sink(probe2)).run()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe2)).run(), 3.seconds) }
probe2.expectErrorOrSubscriptionFollowedByError()
val probe3 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm3 = binding.connections.to(Sink(probe3)).run()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.to(Sink(probe3)).run(), 3.seconds) }
probe3.expectErrorOrSubscriptionFollowedByError()
an[BindFailedException] shouldBe thrownBy { Await.result(binding.localAddress(mm2), 1.second) }
an[BindFailedException] shouldBe thrownBy { Await.result(binding.localAddress(mm3), 1.second) }
// The unbind should NOT fail even though the bind failed.
Await.result(binding.unbind(mm2), 1.second)
Await.result(binding.unbind(mm3), 1.second)
// Now unbind the first
Await.result(binding.unbind(mm1), 1.second)
Await.result(b1.unbind(), 1.second)
probe1.expectComplete()
if (!akka.util.Helpers.isWindows) {
val probe4 = StreamTestKit.SubscriberProbe[Http.IncomingConnection]()
val mm4 = binding.connections.to(Sink(probe4)).run()
// Bind succeeded, we have a local address
val b2 = Await.result(binding.to(Sink(probe4)).run(), 3.seconds)
probe4.expectSubscription()
// Bind succeeded, we have a local address
Await.result(binding.localAddress(mm4), 1.second)
// clean up
Await.result(binding.unbind(mm4), 1.second)
Await.result(b2.unbind(), 1.second)
}
}
@ -134,7 +123,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
private val HttpRequest(POST, uri, List(Accept(Seq(MediaRanges.`*/*`)), Host(_, _), `User-Agent`(_)),
Chunked(`chunkedContentType`, chunkStream), HttpProtocols.`HTTP/1.1`) = serverIn.expectNext()
uri shouldEqual Uri(s"http://$hostname:$port/chunked")
Await.result(chunkStream.grouped(4).runWith(Sink.head), 100.millis) shouldEqual chunks
Await.result(chunkStream.grouped(4).runWith(Sink.head()), 100.millis) shouldEqual chunks
val serverOutSub = serverOut.expectSubscription()
serverOutSub.expectRequest()
@ -144,7 +133,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
clientInSub.request(1)
val HttpResponse(StatusCodes.PartialContent, List(Age(42), Server(_), Date(_)),
Chunked(`chunkedContentType`, chunkStream2), HttpProtocols.`HTTP/1.1`) = clientIn.expectNext()
Await.result(chunkStream2.grouped(1000).runWith(Sink.head), 100.millis) shouldEqual chunks
Await.result(chunkStream2.grouped(1000).runWith(Sink.head()), 100.millis) shouldEqual chunks
clientOutSub.sendComplete()
serverInSub.request(1) // work-around for #16552
@ -194,7 +183,7 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
val settings = configOverrides.toOption.map(ServerSettings.apply)
val binding = Http().bind(hostname, port, settings = settings)
val probe = StreamTestKit.SubscriberProbe[Http.IncomingConnection]
binding.connections.runWith(Sink(probe))
binding.runWith(Sink(probe))
probe
}
val connSourceSub = connSource.expectSubscription()
@ -202,23 +191,35 @@ class ClientServerSpec extends WordSpec with Matchers with BeforeAndAfterAll {
def openNewClientConnection(settings: Option[ClientConnectionSettings] = None): (PublisherProbe[HttpRequest], SubscriberProbe[HttpResponse]) = {
val requestPublisherProbe = StreamTestKit.PublisherProbe[HttpRequest]()
val responseSubscriberProbe = StreamTestKit.SubscriberProbe[HttpResponse]()
val connection = Http().outgoingConnection(hostname, port, settings = settings)
val connectionFuture = Source(requestPublisherProbe)
.viaMat(Http().outgoingConnection(hostname, port, settings = settings))(Keep.right)
.to(Sink(responseSubscriberProbe)).run()
val connection = Await.result(connectionFuture, 3.seconds)
connection.remoteAddress.getHostName shouldEqual hostname
connection.remoteAddress.getPort shouldEqual port
Source(requestPublisherProbe).via(connection.flow).runWith(Sink(responseSubscriberProbe))
requestPublisherProbe -> responseSubscriberProbe
}
def acceptConnection(): (SubscriberProbe[HttpRequest], PublisherProbe[HttpResponse]) = {
connSourceSub.request(1)
val incomingConnection = connSource.expectNext()
val sink = PublisherSink[HttpRequest]()
val source = SubscriberSource[HttpResponse]()
val mm = incomingConnection.handleWith(Flow(sink, source))
val sink = Sink.publisher[HttpRequest]
val source = Source.subscriber[HttpResponse]
val handler = Flow(sink, source)(Keep.both) { implicit b
(snk, src)
(snk.inlet, src.outlet)
}
val (pub, sub) = incomingConnection.handleWith(handler)
val requestSubscriberProbe = StreamTestKit.SubscriberProbe[HttpRequest]()
val responsePublisherProbe = StreamTestKit.PublisherProbe[HttpResponse]()
mm.get(sink).subscribe(requestSubscriberProbe)
responsePublisherProbe.subscribe(mm.get(source))
pub.subscribe(requestSubscriberProbe)
responsePublisherProbe.subscribe(sub)
requestSubscriberProbe -> responsePublisherProbe
}

View file

@ -8,7 +8,7 @@ import com.typesafe.config.{ Config, ConfigFactory }
import scala.util.{ Failure, Success }
import akka.actor.ActorSystem
import akka.stream.ActorFlowMaterializer
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.scaladsl.{ Keep, Sink, Source }
import akka.http.model._
object TestClient extends App {
@ -25,7 +25,7 @@ object TestClient extends App {
println(s"Fetching HTTP server version of host `$host` ...")
val connection = Http().outgoingConnection(host)
val result = Source.single(HttpRequest()).via(connection.flow).runWith(Sink.head)
val result = Source.single(HttpRequest()).via(connection).runWith(Sink.head())
result.map(_.header[headers.Server]) onComplete {
case Success(res) println(s"$host is running ${res mkString ", "}")

View file

@ -19,14 +19,12 @@ object TestServer extends App {
implicit val system = ActorSystem("ServerTest", testConf)
implicit val fm = ActorFlowMaterializer()
val binding = Http().bind(interface = "localhost", port = 8080)
binding startHandlingWithSyncHandler {
val binding = Http().bindAndStartHandlingWithSyncHandler({
case HttpRequest(GET, Uri.Path("/"), _, _, _) index
case HttpRequest(GET, Uri.Path("/ping"), _, _, _) HttpResponse(entity = "PONG!")
case HttpRequest(GET, Uri.Path("/crash"), _, _, _) sys.error("BOOM!")
case _: HttpRequest HttpResponse(404, entity = "Unknown resource!")
}
}, interface = "localhost", port = 8080)
println(s"Server online at http://localhost:8080")
println("Press RETURN to stop...")

View file

@ -358,9 +358,16 @@ class HttpClientSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val (netOut, netIn) = {
val netOut = StreamTestKit.SubscriberProbe[ByteString]
val netIn = StreamTestKit.PublisherProbe[ByteString]
val clientFlow = HttpClient.transportToConnectionClientFlow(
Flow(Sink(netOut), Source(netIn)), remoteAddress, settings, NoLogging)
Source(requests).via(clientFlow).runWith(Sink(responses))
FlowGraph.closed(HttpClient.clientBlueprint(remoteAddress, settings, NoLogging)) { implicit b
client
import FlowGraph.Implicits._
Source(netIn) ~> client.bytesIn
client.bytesOut ~> Sink(netOut)
Source(requests) ~> client.httpRequests
client.httpResponses ~> Sink(responses)
}.run()
netOut -> netIn
}

View file

@ -233,7 +233,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
val parser = newParser
val result = multiParse(newParser)(Seq(prep(start + manyChunks)))
val HttpEntity.Chunked(_, chunks) = result.head.right.get.req.entity
val strictChunks = chunks.grouped(100000).runWith(Sink.head).awaitResult(awaitAtMost)
val strictChunks = chunks.grouped(100000).runWith(Sink.head()).awaitResult(awaitAtMost)
strictChunks.size shouldEqual numChunks
}
}
@ -462,7 +462,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
}
.flatten(FlattenStrategy.concat)
.map(strictEqualify)
.grouped(100000).runWith(Sink.head)
.grouped(100000).runWith(Sink.head())
.awaitResult(awaitAtMost)
protected def parserSettings: ParserSettings = ParserSettings(system)
@ -474,12 +474,12 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(awaitAtMost)
}
private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Seq[ChunkStreamPart]] =
data.grouped(100000).runWith(Sink.head)
private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Seq[ChunkStreamPart]] =
data.grouped(100000).runWith(Sink.head())
.fast.recover { case _: NoSuchElementException Nil }
def prep(response: String) = response.stripMarginWithNewline("\r\n")
}
def source[T](elems: T*): Source[T] = Source(elems.toList)
def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
}

View file

@ -279,7 +279,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
}
.flatten(FlattenStrategy.concat)
.map(strictEqualify)
.grouped(100000).runWith(Sink.head)
.grouped(100000).runWith(Sink.head())
Await.result(future, 500.millis)
}
@ -297,13 +297,13 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(250.millis)
}
private def compactEntityChunks(data: Source[ChunkStreamPart]): Future[Source[ChunkStreamPart]] =
data.grouped(100000).runWith(Sink.head)
private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Source[ChunkStreamPart, Unit]] =
data.grouped(100000).runWith(Sink.head())
.fast.map(source(_: _*))
.fast.recover { case _: NoSuchElementException source() }
def prep(response: String) = response.stripMarginWithNewline("\r\n")
def source[T](elems: T*): Source[T] = Source(elems.toList)
def source[T](elems: T*): Source[T, Unit] = Source(elems.toList)
}
}

View file

@ -21,6 +21,7 @@ import akka.stream.ActorFlowMaterializer
import akka.stream.impl.SynchronousIterablePublisher
import HttpEntity._
import HttpMethods._
import akka.util.ByteString
class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
val testConf: Config = ConfigFactory.parseString("""
@ -255,8 +256,8 @@ class RequestRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
val renderer = newRenderer
val byteStringSource = Await.result(Source.single(RequestRenderingContext(request, serverAddress)).
section(name("renderer"))(_.transform(() renderer)).
runWith(Sink.head), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String)
runWith(Sink.head()), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head()).map(_.reduceLeft(_ ++ _).utf8String)
Await.result(future, 250.millis)
}
}

View file

@ -413,8 +413,8 @@ class ResponseRendererSpec extends FreeSpec with Matchers with BeforeAndAfterAll
val renderer = newRenderer
val byteStringSource = Await.result(Source.single(ctx).
section(name("renderer"))(_.transform(() renderer)).
runWith(Sink.head), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head).map(_.reduceLeft(_ ++ _).utf8String)
runWith(Sink.head()), 1.second)
val future = byteStringSource.grouped(1000).runWith(Sink.head()).map(_.reduceLeft(_ ++ _).utf8String)
Await.result(future, 250.millis) -> renderer.isComplete
}

View file

@ -659,8 +659,16 @@ class HttpServerSpec extends AkkaSpec("akka.loggers = []\n akka.loglevel = OFF")
val (netIn, netOut) = {
val netIn = StreamTestKit.PublisherProbe[ByteString]
val netOut = StreamTestKit.SubscriberProbe[ByteString]
val transportFlow = HttpServer.serverFlowToTransport(Flow(Sink(requests), Source(responses)), settings, NoLogging)
Source(netIn).via(transportFlow).runWith(Sink(netOut))
FlowGraph.closed(HttpServer.serverBlueprint(settings, NoLogging)) { implicit b
server
import FlowGraph.Implicits._
Source(netIn) ~> server.bytesIn
server.bytesOut ~> Sink(netOut)
server.httpRequests ~> Sink(requests)
Source(responses) ~> server.httpResponses
}.run()
netIn -> netOut
}

View file

@ -107,7 +107,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
def collectBytesTo(bytes: ByteString*): Matcher[HttpEntity] =
equal(bytes.toVector).matcher[Seq[ByteString]].compose { entity
val future = entity.dataBytes.grouped(1000).runWith(Sink.head)
val future = entity.dataBytes.grouped(1000).runWith(Sink.head())
Await.result(future, 250.millis)
}
@ -120,7 +120,7 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
Await.result(transformed.toStrict(250.millis), 250.millis)
}
def duplicateBytesTransformer(): Flow[ByteString, ByteString] =
def duplicateBytesTransformer(): Flow[ByteString, ByteString, Unit] =
Flow[ByteString].transform(() StreamUtils.byteStringTransformer(doubleChars, () trailer))
def trailer: ByteString = ByteString("--dup")