!str #16902: Unify stream internal representation
also =str #16912: Fix StreamTcpSpec flakiness
This commit is contained in:
parent
cac9c9f2fb
commit
8d77fa8b29
230 changed files with 7814 additions and 9596 deletions
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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 ///////////////////
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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() = {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
})
|
||||
|
|
|
|||
|
|
@ -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. */
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 ", "}")
|
||||
|
|
|
|||
|
|
@ -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...")
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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")
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue