!htc #17039, #16933 change default materialization placeholder type from Unit to Any

This commit is contained in:
Mathias 2015-04-02 16:54:12 +02:00
parent 85c0571620
commit a4c279c9c5
21 changed files with 90 additions and 86 deletions

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

@ -110,7 +110,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
* +------+ * +------+
* }}} * }}}
*/ */
type ServerLayer = BidiFlow[HttpResponse, ByteString, ByteString, HttpRequest, Any] type ServerLayer = BidiFlow[HttpResponse, ByteString, ByteString, HttpRequest, Unit]
/** /**
* Constructs a [[ServerLayer]] stage using the configured default [[ServerSettings]]. * Constructs a [[ServerLayer]] stage using the configured default [[ServerSettings]].
@ -158,7 +158,7 @@ class HttpExt(config: Config)(implicit system: ActorSystem) extends akka.actor.E
* +------+ * +------+
* }}} * }}}
*/ */
type ClientLayer = BidiFlow[HttpRequest, ByteString, ByteString, HttpResponse, Any] type ClientLayer = BidiFlow[HttpRequest, ByteString, ByteString, HttpResponse, Unit]
/** /**
* Constructs a [[ClientLayer]] stage using the configured default [[ClientConnectionSettings]]. * Constructs a [[ClientLayer]] stage using the configured default [[ClientConnectionSettings]].
@ -200,7 +200,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
case class IncomingConnection( case class IncomingConnection(
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
flow: Flow[HttpResponse, HttpRequest, Any]) { flow: Flow[HttpResponse, HttpRequest, Unit]) {
/** /**
* Handles the connection with the given flow, which is materialized exactly once * Handles the connection with the given flow, which is materialized exactly once

View file

@ -9,7 +9,7 @@ import scala.annotation.tailrec
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.stream.scaladsl.OperationAttributes._ import akka.stream.scaladsl.OperationAttributes._
import akka.stream.stage.{ Context, PushPullStage } import akka.stream.stage.{ Context, PushPullStage }
import akka.stream.scaladsl.Source import akka.stream.scaladsl.{ Keep, Source }
import akka.util.ByteString import akka.util.ByteString
import akka.http.model.parser.CharacterClasses import akka.http.model.parser.CharacterClasses
import akka.http.util.identityFunc import akka.http.util.identityFunc
@ -126,7 +126,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
emit(RequestStart(method, uri, protocol, allHeaders, createEntity, expect100continue, closeAfterResponseCompletion)) emit(RequestStart(method, uri, protocol, allHeaders, createEntity, expect100continue, closeAfterResponseCompletion))
} }
def expect100continueHandling[T]: Source[T, Unit] Source[T, Unit] = def expect100continueHandling[T, Mat]: Source[T, Mat] Source[T, Mat] =
if (expect100continue) { if (expect100continue) {
_.section(name("expect100continueTrigger"))(_.transform(() new PushPullStage[T, T] { _.section(name("expect100continueTrigger"))(_.transform(() new PushPullStage[T, T] {
private var oneHundredContinueSent = false private var oneHundredContinueSent = false

View file

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

View file

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

View file

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

View file

@ -29,24 +29,24 @@ private object RenderSupport {
val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk) val defaultLastChunkBytes: ByteString = renderChunk(HttpEntity.LastChunk)
def CancelSecond[T](first: Source[T, _], second: Source[T, _]): Source[T, Unit] = { def CancelSecond[T, Mat](first: Source[T, Mat], second: Source[T, Any]): Source[T, Mat] = {
Source(first) { implicit b Source(first) { implicit b
frst frst
import FlowGraph.Implicits._ import FlowGraph.Implicits._
second ~> Sink.cancelled second ~> Sink.cancelled
frst.outlet frst.outlet
}.mapMaterialized((_) ()) }
} }
def renderEntityContentType(r: Rendering, entity: HttpEntity) = def renderEntityContentType(r: Rendering, entity: HttpEntity) =
if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf if (entity.contentType != ContentTypes.NoContentType) r ~~ headers.`Content-Type` ~~ entity.contentType ~~ CrLf
else r else r
def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString, Unit], def renderByteStrings(r: ByteStringRendering, entityBytes: Source[ByteString, Any],
skipEntity: Boolean = false): Source[ByteString, Unit] = { skipEntity: Boolean = false): Source[ByteString, Any] = {
val messageStart = Source.single(r.get) val messageStart = Source.single(r.get)
val messageBytes = val messageBytes =
if (!skipEntity) (messageStart ++ entityBytes).mapMaterialized((_) ()) if (!skipEntity) (messageStart ++ entityBytes).mapMaterialized(_ ())
else CancelSecond(messageStart, entityBytes) else CancelSecond(messageStart, entityBytes)
messageBytes messageBytes
} }

View file

@ -26,7 +26,7 @@ private[http] object HttpServerBluePrint {
type ServerShape = BidiShape[HttpResponse, ByteString, ByteString, HttpRequest] type ServerShape = BidiShape[HttpResponse, ByteString, ByteString, HttpRequest]
def apply(settings: ServerSettings, log: LoggingAdapter)(implicit mat: FlowMaterializer): Graph[ServerShape, Any] = { def apply(settings: ServerSettings, log: LoggingAdapter)(implicit mat: FlowMaterializer): Graph[ServerShape, Unit] = {
import settings._ import settings._
// the initial header parser we initially use for every connection, // the initial header parser we initially use for every connection,

View file

@ -35,7 +35,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
/** /**
* A stream of the data of this entity. * A stream of the data of this entity.
*/ */
def dataBytes: Source[ByteString, Unit] def dataBytes: Source[ByteString, Any]
/** /**
* Collects all possible parts and returns a potentially future Strict entity for easier processing. * Collects all possible parts and returns a potentially future Strict entity for easier processing.
@ -72,7 +72,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
* This method may only throw an exception if the `transformer` function throws an exception while creating the transformer. * 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. * Any other errors are reported through the new entity data stream.
*/ */
def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): HttpEntity def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): HttpEntity
/** /**
* Creates a copy of this HttpEntity with the `contentType` overridden with the given one. * Creates a copy of this HttpEntity with the `contentType` overridden with the given one.
@ -80,7 +80,7 @@ sealed trait HttpEntity extends japi.HttpEntity {
def withContentType(contentType: ContentType): HttpEntity def withContentType(contentType: ContentType): HttpEntity
/** Java API */ /** Java API */
def getDataBytes: Source[ByteString, Unit] = dataBytes def getDataBytes: Source[ByteString, _] = dataBytes
// default implementations, should be overridden // default implementations, should be overridden
def isCloseDelimited: Boolean = false def isCloseDelimited: Boolean = false
@ -97,13 +97,13 @@ sealed trait BodyPartEntity extends HttpEntity with japi.BodyPartEntity {
sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity { sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity {
def withContentType(contentType: ContentType): RequestEntity def withContentType(contentType: ContentType): RequestEntity
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): RequestEntity override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): RequestEntity
} }
/* An entity that can be used for responses */ /* An entity that can be used for responses */
sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity { sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity {
def withContentType(contentType: ContentType): ResponseEntity def withContentType(contentType: ContentType): ResponseEntity
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): ResponseEntity override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): ResponseEntity
} }
/* An entity that can be used for requests, responses, and body parts */ /* An entity that can be used for requests, responses, and body parts */
sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity { sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity {
@ -114,7 +114,7 @@ sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity wit
* Transforms this' entities data bytes with a transformer that will produce exactly the number of bytes given as * Transforms this' entities data bytes with a transformer that will produce exactly the number of bytes given as
* ``newContentLength``. * ``newContentLength``.
*/ */
def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, Any]): UniversalEntity
} }
object HttpEntity { object HttpEntity {
@ -127,7 +127,7 @@ object HttpEntity {
if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes)) if (bytes.length == 0) empty(contentType) else apply(contentType, ByteString(bytes))
def apply(contentType: ContentType, data: ByteString): Strict = def apply(contentType: ContentType, data: ByteString): Strict =
if (data.isEmpty) empty(contentType) else Strict(contentType, data) if (data.isEmpty) empty(contentType) else Strict(contentType, data)
def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString, Unit]): UniversalEntity = def apply(contentType: ContentType, contentLength: Long, data: Source[ByteString, Any]): UniversalEntity =
if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data) if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data)
def apply(contentType: ContentType, file: File): UniversalEntity = { def apply(contentType: ContentType, file: File): UniversalEntity = {
@ -160,11 +160,11 @@ object HttpEntity {
override def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit fm: FlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): MessageEntity = override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): MessageEntity =
Chunked.fromData(contentType, Source.single(data).via(transformer)) Chunked.fromData(contentType, Source.single(data).via(transformer))
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity = override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, Any]): UniversalEntity =
Default(contentType, newContentLength, Source.single(data).via(transformer)) Default(contentType, newContentLength, Source.single(data) via transformer)
def withContentType(contentType: ContentType): Strict = def withContentType(contentType: ContentType): Strict =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
@ -177,20 +177,20 @@ object HttpEntity {
*/ */
final case class Default(contentType: ContentType, final case class Default(contentType: ContentType,
contentLength: Long, contentLength: Long,
data: Source[ByteString, Unit]) data: Source[ByteString, Any])
extends japi.HttpEntityDefault with UniversalEntity { extends japi.HttpEntityDefault with UniversalEntity {
require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)") require(contentLength > 0, "contentLength must be positive (use `HttpEntity.empty(contentType)` for empty entities)")
def isKnownEmpty = false def isKnownEmpty = false
override def isDefault: Boolean = true override def isDefault: Boolean = true
def dataBytes: Source[ByteString, Unit] = data def dataBytes: Source[ByteString, Any] = data
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): Chunked = override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): Chunked =
Chunked.fromData(contentType, data.viaMat(transformer)(Keep.left)) Chunked.fromData(contentType, data via transformer)
override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, _]): UniversalEntity = override def transformDataBytes(newContentLength: Long, transformer: Flow[ByteString, ByteString, Any]): UniversalEntity =
Default(contentType, newContentLength, data.viaMat(transformer)(Keep.left)) Default(contentType, newContentLength, data via transformer)
def withContentType(contentType: ContentType): Default = def withContentType(contentType: ContentType): Default =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
@ -205,11 +205,11 @@ object HttpEntity {
*/ */
private[http] sealed trait WithoutKnownLength extends HttpEntity { private[http] sealed trait WithoutKnownLength extends HttpEntity {
def contentType: ContentType def contentType: ContentType
def data: Source[ByteString, Unit] def data: Source[ByteString, Any]
def isKnownEmpty = data eq Source.empty def isKnownEmpty = data eq Source.empty
def dataBytes: Source[ByteString, Unit] = data def dataBytes: Source[ByteString, Any] = data
} }
/** /**
@ -217,7 +217,7 @@ object HttpEntity {
* The content-length of such responses is unknown at the time the response headers have been received. * 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. * Note that this type of HttpEntity can only be used for HttpResponses.
*/ */
final case class CloseDelimited(contentType: ContentType, data: Source[ByteString, Unit]) final case class CloseDelimited(contentType: ContentType, data: Source[ByteString, Any])
extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength { extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength {
type Self = CloseDelimited type Self = CloseDelimited
@ -225,8 +225,8 @@ object HttpEntity {
def withContentType(contentType: ContentType): CloseDelimited = def withContentType(contentType: ContentType): CloseDelimited =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): CloseDelimited = override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): CloseDelimited =
HttpEntity.CloseDelimited(contentType, data.viaMat(transformer)(Keep.left)) HttpEntity.CloseDelimited(contentType, data via transformer)
override def productPrefix = "HttpEntity.CloseDelimited" override def productPrefix = "HttpEntity.CloseDelimited"
} }
@ -235,15 +235,15 @@ object HttpEntity {
* The model for the entity of a BodyPart with an indefinite length. * The model for the entity of a BodyPart with an indefinite length.
* Note that this type of HttpEntity can only be used for BodyParts. * Note that this type of HttpEntity can only be used for BodyParts.
*/ */
final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString, Unit]) final case class IndefiniteLength(contentType: ContentType, data: Source[ByteString, Any])
extends japi.HttpEntityIndefiniteLength with BodyPartEntity with WithoutKnownLength { extends japi.HttpEntityIndefiniteLength with BodyPartEntity with WithoutKnownLength {
override def isIndefiniteLength: Boolean = true override def isIndefiniteLength: Boolean = true
def withContentType(contentType: ContentType): IndefiniteLength = def withContentType(contentType: ContentType): IndefiniteLength =
if (contentType == this.contentType) this else copy(contentType = contentType) if (contentType == this.contentType) this else copy(contentType = contentType)
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): IndefiniteLength = override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): IndefiniteLength =
HttpEntity.IndefiniteLength(contentType, data.viaMat(transformer)(Keep.left)) HttpEntity.IndefiniteLength(contentType, data via transformer)
override def productPrefix = "HttpEntity.IndefiniteLength" override def productPrefix = "HttpEntity.IndefiniteLength"
} }
@ -251,23 +251,22 @@ object HttpEntity {
/** /**
* The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`). * The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`).
*/ */
final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart, Unit]) final case class Chunked(contentType: ContentType, chunks: Source[ChunkStreamPart, Any])
extends japi.HttpEntityChunked with MessageEntity { extends japi.HttpEntityChunked with MessageEntity {
def isKnownEmpty = chunks eq Source.empty def isKnownEmpty = chunks eq Source.empty
override def isChunked: Boolean = true override def isChunked: Boolean = true
def dataBytes: Source[ByteString, Unit] = def dataBytes: Source[ByteString, Any] = chunks.map(_.data).filter(_.nonEmpty)
chunks.map(_.data).filter(_.nonEmpty)
override def transformDataBytes(transformer: Flow[ByteString, ByteString, _]): Chunked = { override def transformDataBytes(transformer: Flow[ByteString, ByteString, Any]): Chunked = {
val newData = val newData =
chunks.map { chunks.map {
case Chunk(data, "") data case Chunk(data, "") data
case LastChunk("", Nil) ByteString.empty case LastChunk("", Nil) ByteString.empty
case _ case _
throw new IllegalArgumentException("Chunked.transformDataBytes not allowed for chunks with metadata") throw new IllegalArgumentException("Chunked.transformDataBytes not allowed for chunks with metadata")
}.viaMat(transformer)(Keep.left) } via transformer
Chunked.fromData(contentType, newData) Chunked.fromData(contentType, newData)
} }
@ -278,14 +277,14 @@ object HttpEntity {
override def productPrefix = "HttpEntity.Chunked" override def productPrefix = "HttpEntity.Chunked"
/** Java API */ /** Java API */
def getChunks: Source[japi.ChunkStreamPart, Unit] = chunks.asInstanceOf[Source[japi.ChunkStreamPart, Unit]] def getChunks: Source[japi.ChunkStreamPart, Any] = chunks.asInstanceOf[Source[japi.ChunkStreamPart, Any]]
} }
object Chunked { object Chunked {
/** /**
* Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given * Returns a ``Chunked`` entity where one Chunk is produced for every non-empty ByteString of the given
* ``Publisher[ByteString]``. * ``Publisher[ByteString]``.
*/ */
def fromData(contentType: ContentType, chunks: Source[ByteString, Unit]): Chunked = def fromData(contentType: ContentType, chunks: Source[ByteString, Any]): Chunked =
Chunked(contentType, chunks.collect[ChunkStreamPart] { Chunked(contentType, chunks.collect[ChunkStreamPart] {
case b: ByteString if b.nonEmpty Chunk(b) case b: ByteString if b.nonEmpty Chunk(b)
}) })

View file

@ -143,6 +143,9 @@ final case class HttpRequest(method: HttpMethod = HttpMethods.GET,
/** /**
* Resolve this request's URI according to the logic defined at * Resolve this request's URI according to the logic defined at
* http://tools.ietf.org/html/rfc7230#section-5.5 * http://tools.ietf.org/html/rfc7230#section-5.5
*
* Throws an [[IllegalUriException]] if the URI is relative and the `headers` don't
* include a valid [[Host]] header or if URI authority and [[Host]] header don't match.
*/ */
def effectiveUri(securedConnection: Boolean, defaultHostHeader: Host = Host.empty): Uri = def effectiveUri(securedConnection: Boolean, defaultHostHeader: Host = Host.empty): Uri =
HttpRequest.effectiveUri(uri, headers, securedConnection, defaultHostHeader) HttpRequest.effectiveUri(uri, headers, securedConnection, defaultHostHeader)
@ -263,6 +266,9 @@ object HttpRequest {
/** /**
* Determines the effective request URI according to the logic defined at * Determines the effective request URI according to the logic defined at
* http://tools.ietf.org/html/rfc7230#section-5.5 * http://tools.ietf.org/html/rfc7230#section-5.5
*
* Throws an [[IllegalUriException]] if the URI is relative and the `headers` don't
* include a valid [[Host]] header or if URI authority and [[Host]] header don't match.
*/ */
def effectiveUri(uri: Uri, headers: immutable.Seq[HttpHeader], securedConnection: Boolean, defaultHostHeader: Host): Uri = { def effectiveUri(uri: Uri, headers: immutable.Seq[HttpHeader], securedConnection: Boolean, defaultHostHeader: Host): Uri = {
val hostHeader = headers.collectFirst { case x: Host x } val hostHeader = headers.collectFirst { case x: Host x }

View file

@ -17,7 +17,7 @@ import FastFuture._
trait Multipart { trait Multipart {
def mediaType: MultipartMediaType def mediaType: MultipartMediaType
def parts: Source[Multipart.BodyPart, Unit] def parts: Source[Multipart.BodyPart, Any]
/** /**
* Converts this content into its strict counterpart. * Converts this content into its strict counterpart.
@ -56,7 +56,7 @@ object Multipart {
} }
} }
private def strictify[BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](parts: Source[BP, Unit])(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, Any])(f: BP Future[BPS])(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[Vector[BPS]] =
// TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed // TODO: move to Vector `:+` when https://issues.scala-lang.org/browse/SI-8930 is fixed
parts.runFold(new VectorBuilder[Future[BPS]]) { parts.runFold(new VectorBuilder[Future[BPS]]) {
case (builder, part) builder += f(part) case (builder, part) builder += f(part)
@ -69,27 +69,27 @@ object Multipart {
*/ */
sealed abstract class General extends Multipart { sealed abstract class General extends Multipart {
def mediaType: MultipartMediaType def mediaType: MultipartMediaType
def parts: Source[General.BodyPart, Unit] def parts: Source[General.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[General.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[General.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _)) strictify(parts)(_.toStrict(timeout)).fast.map(General.Strict(mediaType, _))
} }
object General { object General {
def apply(mediaType: MultipartMediaType, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector) def apply(mediaType: MultipartMediaType, parts: BodyPart.Strict*): Strict = Strict(mediaType, parts.toVector)
def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart, Unit]): General = def apply(_mediaType: MultipartMediaType, _parts: Source[BodyPart, Any]): General =
new General { new General {
def mediaType = _mediaType def mediaType = _mediaType
def parts = _parts def parts = _parts
override def toString = s"General($mediaType, $parts)" override def toString = s"General($mediaType, $parts)"
} }
def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart, Unit])] = Some(value.mediaType -> value.parts) def unapply(value: General): Option[(MultipartMediaType, Source[BodyPart, Any])] = Some(value.mediaType -> value.parts)
/** /**
* Strict [[General]]. * Strict [[General]].
*/ */
case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict { case class Strict(mediaType: MultipartMediaType, strictParts: immutable.Seq[BodyPart.Strict]) extends General with Multipart.Strict {
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts) def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "General.Strict" override def productPrefix = "General.Strict"
@ -148,7 +148,7 @@ object Multipart {
*/ */
sealed abstract class FormData extends Multipart { sealed abstract class FormData extends Multipart {
def mediaType = MediaTypes.`multipart/form-data` def mediaType = MediaTypes.`multipart/form-data`
def parts: Source[FormData.BodyPart, Unit] def parts: Source[FormData.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[FormData.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[FormData.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_)) strictify(parts)(_.toStrict(timeout)).fast.map(FormData.Strict(_))
} }
@ -159,7 +159,7 @@ object Multipart {
fields.map { case (name, entity) BodyPart.Strict(name, entity) }(collection.breakOut) fields.map { case (name, entity) BodyPart.Strict(name, entity) }(collection.breakOut)
} }
def apply(_parts: Source[BodyPart, Unit]): FormData = new FormData { def apply(_parts: Source[BodyPart, Any]): FormData = new FormData {
def parts = _parts def parts = _parts
override def toString = s"FormData($parts)" override def toString = s"FormData($parts)"
} }
@ -168,7 +168,7 @@ object Multipart {
* Strict [[FormData]]. * Strict [[FormData]].
*/ */
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict { case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends FormData with Multipart.Strict {
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts) def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "FormData.Strict" override def productPrefix = "FormData.Strict"
@ -223,14 +223,14 @@ object Multipart {
*/ */
sealed abstract class ByteRanges extends Multipart { sealed abstract class ByteRanges extends Multipart {
def mediaType = MediaTypes.`multipart/byteranges` def mediaType = MediaTypes.`multipart/byteranges`
def parts: Source[ByteRanges.BodyPart, Unit] def parts: Source[ByteRanges.BodyPart, Any]
def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[ByteRanges.Strict] = def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer): Future[ByteRanges.Strict] =
strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_)) strictify(parts)(_.toStrict(timeout)).fast.map(ByteRanges.Strict(_))
} }
object ByteRanges { object ByteRanges {
def apply(parts: BodyPart.Strict*): Strict = Strict(parts.toVector) def apply(parts: BodyPart.Strict*): Strict = Strict(parts.toVector)
def apply(_parts: Source[BodyPart, Unit]): ByteRanges = def apply(_parts: Source[BodyPart, Any]): ByteRanges =
new ByteRanges { new ByteRanges {
def parts = _parts def parts = _parts
override def toString = s"ByteRanges($parts)" override def toString = s"ByteRanges($parts)"
@ -240,7 +240,7 @@ object Multipart {
* Strict [[ByteRanges]]. * Strict [[ByteRanges]].
*/ */
case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict { case class Strict(strictParts: immutable.Seq[BodyPart.Strict]) extends ByteRanges with Multipart.Strict {
def parts: Source[BodyPart.Strict, Unit] = Source(strictParts) def parts: Source[BodyPart.Strict, Any] = Source(strictParts)
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) = override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
FastFuture.successful(this) FastFuture.successful(this)
override def productPrefix = "ByteRanges.Strict" override def productPrefix = "ByteRanges.Strict"

View file

@ -128,7 +128,7 @@ private[http] object StreamUtils {
* Applies a sequence of transformers on one source and returns a sequence of sources with the result. The input source * 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. * will only be traversed once.
*/ */
def transformMultiple(input: Source[ByteString, Unit], transformers: immutable.Seq[Flow[ByteString, ByteString, _]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString, Unit]] = def transformMultiple(input: Source[ByteString, Any], transformers: immutable.Seq[Flow[ByteString, ByteString, Any]])(implicit materializer: FlowMaterializer): immutable.Seq[Source[ByteString, Any]] =
transformers match { transformers match {
case Nil Nil case Nil Nil
case Seq(one) Vector(input.via(one)) case Seq(one) Vector(input.via(one))
@ -137,12 +137,11 @@ private[http] object StreamUtils {
val sources = transformers.map { flow val sources = transformers.map { flow
// Doubly wrap to ensure that subscription to the running publisher happens before the final sources // Doubly wrap to ensure that subscription to the running publisher happens before the final sources
// are exposed, so there is no race // are exposed, so there is no race
Source(Source(fanoutPub).via(flow).runWith(Sink.publisher)) Source(Source(fanoutPub).viaMat(flow)(Keep.right).runWith(Sink.publisher))
} }
// The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed // The fanout publisher must be wired to the original source after all fanout subscribers have been subscribed
input.runWith(Sink(fanoutSub)) input.runWith(Sink(fanoutSub))
sources sources
} }
def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity = def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity =

View file

@ -41,13 +41,13 @@ package object util {
private[http] implicit def enhanceByteStrings[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] = private[http] implicit def enhanceByteStrings[Mat](byteStrings: Source[ByteString, Mat]): EnhancedByteStringSource[Mat] =
new EnhancedByteStringSource(byteStrings) new EnhancedByteStringSource(byteStrings)
private[http] implicit class SourceWithHeadAndTail[T, Mat](val underlying: Source[Source[T, Unit], Mat]) extends AnyVal { private[http] implicit class SourceWithHeadAndTail[T, Mat](val underlying: Source[Source[T, Any], Mat]) extends AnyVal {
def headAndTail: Source[(T, Source[T, Unit]), Mat] = def headAndTail: Source[(T, Source[T, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } } underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)
} }
private[http] implicit class FlowWithHeadAndTail[In, Out, Mat](val underlying: Flow[In, Source[Out, Unit], Mat]) extends AnyVal { private[http] implicit class FlowWithHeadAndTail[In, Out, Mat](val underlying: Flow[In, Source[Out, Any], Mat]) extends AnyVal {
def headAndTail: Flow[In, (Out, Source[Out, Unit]), Mat] = def headAndTail: Flow[In, (Out, Source[Out, Unit]), Mat] =
underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } } underlying.map { _.prefixAndTail(1).map { case (prefix, tail) (prefix.head, tail) } }
.flatten(FlattenStrategy.concat) .flatten(FlattenStrategy.concat)

View file

@ -474,7 +474,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(awaitAtMost) case _ entity.toStrict(awaitAtMost)
} }
private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Seq[ChunkStreamPart]] = private def compactEntityChunks(data: Source[ChunkStreamPart, Any]): Future[Seq[ChunkStreamPart]] =
data.grouped(100000).runWith(Sink.head) data.grouped(100000).runWith(Sink.head)
.fast.recover { case _: NoSuchElementException Nil } .fast.recover { case _: NoSuchElementException Nil }

View file

@ -297,7 +297,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
case _ entity.toStrict(250.millis) case _ entity.toStrict(250.millis)
} }
private def compactEntityChunks(data: Source[ChunkStreamPart, Unit]): Future[Source[ChunkStreamPart, Unit]] = private def compactEntityChunks(data: Source[ChunkStreamPart, Any]): Future[Source[ChunkStreamPart, Any]] =
data.grouped(100000).runWith(Sink.head) data.grouped(100000).runWith(Sink.head)
.fast.map(source(_: _*)) .fast.map(source(_: _*))
.fast.recover { case _: NoSuchElementException source() } .fast.recover { case _: NoSuchElementException source() }

View file

@ -54,7 +54,7 @@ trait MultipartUnmarshallers {
def multipartUnmarshaller[T <: Multipart, BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](mediaRange: MediaRange, def multipartUnmarshaller[T <: Multipart, BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](mediaRange: MediaRange,
defaultContentType: ContentType, defaultContentType: ContentType,
createBodyPart: (BodyPartEntity, List[HttpHeader]) BP, createBodyPart: (BodyPartEntity, List[HttpHeader]) BP,
createStreamed: (MultipartMediaType, Source[BP, Unit]) T, createStreamed: (MultipartMediaType, Source[BP, Any]) T,
createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS, createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) BPS,
createStrict: (MultipartMediaType, immutable.Seq[BPS]) T)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] = createStrict: (MultipartMediaType, immutable.Seq[BPS]) T)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] =
Unmarshaller { entity Unmarshaller { entity