Merge pull request #15941 from spray/w/15915-restructure-HttpEntity-hierarchy

!hco #15915 introduce more shades of HttpEntities
This commit is contained in:
Björn Antonsson 2014-09-22 13:57:08 +02:00
commit 17c7df6fdd
41 changed files with 300 additions and 181 deletions

View file

@ -0,0 +1,8 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
/** Marker-interface for entity types that can be used in a body part */
public interface BodyPartEntity extends HttpEntity {}

View file

@ -0,0 +1,70 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
import java.io.File;
import akka.util.ByteString;
import org.reactivestreams.Publisher;
import akka.stream.FlowMaterializer;
import akka.http.model.HttpEntity$;
/** Constructors for HttpEntity instances */
public final class HttpEntities {
private HttpEntities() {}
public static HttpEntityStrict create(String string) {
return HttpEntity$.MODULE$.apply(string);
}
public static HttpEntityStrict create(byte[] bytes) {
return HttpEntity$.MODULE$.apply(bytes);
}
public static HttpEntityStrict create(ByteString bytes) {
return HttpEntity$.MODULE$.apply(bytes);
}
public static HttpEntityStrict create(ContentType contentType, String string) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, string);
}
public static HttpEntityStrict create(ContentType contentType, byte[] bytes) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, bytes);
}
public static HttpEntityStrict create(ContentType contentType, ByteString bytes) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, bytes);
}
public static UniversalEntity create(ContentType contentType, File file) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file);
}
public static HttpEntityDefault create(ContentType contentType, long contentLength, Publisher<ByteString> data) {
return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data);
}
public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Publisher<ByteString> data) {
return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data);
}
public static HttpEntityIndefiniteLength createIndefiniteLength(ContentType contentType, Publisher<ByteString> data) {
return new akka.http.model.HttpEntity.IndefiniteLength((akka.http.model.ContentType) contentType, data);
}
public static HttpEntityChunked createChunked(ContentType contentType, Publisher<ChunkStreamPart> chunks) {
return new akka.http.model.HttpEntity.Chunked(
(akka.http.model.ContentType) contentType,
Util.<ChunkStreamPart, akka.http.model.HttpEntity.ChunkStreamPart>upcastPublisher(chunks));
}
public static HttpEntityChunked createChunked(ContentType contentType, Publisher<ByteString> data, FlowMaterializer materializer) {
return akka.http.model.HttpEntity.Chunked$.MODULE$.fromData(
(akka.http.model.ContentType) contentType,
data, materializer);
}
}

View file

@ -9,8 +9,6 @@ import akka.stream.FlowMaterializer;
import akka.util.ByteString;
import org.reactivestreams.Publisher;
import java.io.File;
/**
* Represents the entity of an Http message. An entity consists of the content-type of the data
* and the actual data itself. Some subtypes of HttpEntity also define the content-length of the
@ -19,16 +17,24 @@ import java.io.File;
* An HttpEntity can be of several kinds:
*
* - HttpEntity.Empty: the statically known empty entity
* - HttpEntityStrict: an entity containing already evaluated ByteString data
* - HttpEntityDefault: the default entity which has a known length and which contains
* a stream of ByteStrings.
* - HttpEntityChunked: represents an entity that is delivered using `Transfer-Encoding: chunked`
* - HttpEntityCloseDelimited: the entity which doesn't have a fixed length but which is delimited by
* - HttpEntityCloseDelimited: an entity which doesn't have a fixed length but which is delimited by
* closing the connection.
* - HttpEntityIndefiniteLength: an entity which doesn't have a fixed length which can be used to construct BodyParts
* with indefinite length
*
* All entity subtypes but HttpEntityCloseDelimited are subtypes of {@link HttpEntityRegular} which
* means they can be used in Http request that disallow close-delimited transfer of the entity.
* Marker-interfaces denote which subclasses can be used in which context:
* - RequestEntity: an entity type that can be used in an HttpRequest
* - ResponseEntity: an entity type that can be used in an HttpResponse
* - BodyPartEntity: an entity type that can be used in a BodyPart
* - UniversalEntity: an entity type that can be used in every context
*
* Use the static constructors in HttpEntities to construct instances.
*/
public abstract class HttpEntity {
public interface HttpEntity {
/**
* Returns the content-type of this entity
*/
@ -45,11 +51,6 @@ public abstract class HttpEntity {
*/
public abstract boolean isKnownEmpty();
/**
* Returns if this entity is a subtype of HttpEntityRegular.
*/
public abstract boolean isRegular();
/**
* Returns if this entity is a subtype of HttpEntityChunked.
*/
@ -65,46 +66,13 @@ public abstract class HttpEntity {
*/
public abstract boolean isCloseDelimited();
/**
* Returns if this entity is a subtype of HttpEntityIndefiniteLength.
*/
public abstract boolean isIndefiniteLength();
/**
* Returns a stream of data bytes this entity consists of.
*/
public abstract Publisher<ByteString> getDataBytes(FlowMaterializer materializer);
public static HttpEntityStrict create(String string) {
return HttpEntity$.MODULE$.apply(string);
}
public static HttpEntityStrict create(byte[] bytes) {
return HttpEntity$.MODULE$.apply(bytes);
}
public static HttpEntityStrict create(ByteString bytes) {
return HttpEntity$.MODULE$.apply(bytes);
}
public static HttpEntityStrict create(ContentType contentType, String string) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, string);
}
public static HttpEntityStrict create(ContentType contentType, byte[] bytes) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, bytes);
}
public static HttpEntityStrict create(ContentType contentType, ByteString bytes) {
return HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, bytes);
}
public static HttpEntityRegular create(ContentType contentType, File file) {
return (HttpEntityRegular) HttpEntity$.MODULE$.apply((akka.http.model.ContentType) contentType, file);
}
public static HttpEntityDefault create(ContentType contentType, long contentLength, Publisher<ByteString> data) {
return new akka.http.model.HttpEntity.Default((akka.http.model.ContentType) contentType, contentLength, data);
}
public static HttpEntityCloseDelimited createCloseDelimited(ContentType contentType, Publisher<ByteString> data) {
return new akka.http.model.HttpEntity.CloseDelimited((akka.http.model.ContentType) contentType, data);
}
public static HttpEntityChunked createChunked(ContentType contentType, Publisher<ChunkStreamPart> chunks) {
return new akka.http.model.HttpEntity.Chunked(
(akka.http.model.ContentType) contentType,
Util.<ChunkStreamPart, akka.http.model.HttpEntity.ChunkStreamPart>upcastPublisher(chunks));
}
public static HttpEntityChunked createChunked(ContentType contentType, Publisher<ByteString> data, FlowMaterializer materializer) {
return akka.http.model.HttpEntity.Chunked$.MODULE$.fromData(
(akka.http.model.ContentType) contentType,
data, materializer);
}
}

View file

@ -10,6 +10,6 @@ import org.reactivestreams.Publisher;
* Represents an entity transferred using `Transfer-Encoding: chunked`. It consists of a
* stream of {@link ChunkStreamPart}.
*/
public abstract class HttpEntityChunked extends HttpEntityRegular {
public abstract class HttpEntityChunked implements RequestEntity, ResponseEntity {
public abstract Publisher<ChunkStreamPart> getChunks();
}

View file

@ -12,6 +12,6 @@ import org.reactivestreams.Publisher;
* determined by closing the underlying connection. Therefore, this entity type is only
* available for Http responses.
*/
public abstract class HttpEntityCloseDelimited extends HttpEntity {
public abstract class HttpEntityCloseDelimited implements ResponseEntity {
public abstract Publisher<ByteString> data();
}

View file

@ -10,7 +10,7 @@ import org.reactivestreams.Publisher;
/**
* The default entity type which has a predetermined length and a stream of data bytes.
*/
public abstract class HttpEntityDefault extends HttpEntityRegular {
public abstract class HttpEntityDefault implements BodyPartEntity, RequestEntity, ResponseEntity {
public abstract long contentLength();
public abstract Publisher<ByteString> data();
}

View file

@ -0,0 +1,15 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
import akka.util.ByteString;
import org.reactivestreams.Publisher;
/**
* Represents an entity without a predetermined content-length to use in a BodyParts.
*/
public abstract class HttpEntityIndefiniteLength implements BodyPartEntity {
public abstract Publisher<ByteString> data();
}

View file

@ -1,10 +0,0 @@
/**
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
/**
* A marker type that denotes HttpEntity subtypes that can be used in Http requests.
*/
public abstract class HttpEntityRegular extends HttpEntity {}

View file

@ -9,6 +9,6 @@ import akka.util.ByteString;
/**
* The entity type which consists of a predefined fixed ByteString of data.
*/
public abstract class HttpEntityStrict extends HttpEntityRegular {
public abstract class HttpEntityStrict implements BodyPartEntity, RequestEntity, ResponseEntity {
public abstract ByteString data();
}

View file

@ -48,7 +48,7 @@ public interface HttpMessage {
/**
* The entity of this message.
*/
HttpEntity entity();
ResponseEntity entity();
public static interface MessageTransformations<Self> {
/**
@ -71,11 +71,6 @@ public interface HttpMessage {
*/
Self removeHeader(String headerName);
/**
* Returns a copy of this message with a new entity.
*/
Self withEntity(HttpEntity entity);
/**
* Returns a copy of this message with a new entity.
*/
@ -110,5 +105,10 @@ public interface HttpMessage {
* Returns a copy of Self message with a new entity.
*/
Self withEntity(ContentType type, File file);
/**
* Returns a copy of Self message with a new entity.
*/
Self withEntity(RequestEntity entity);
}
}

View file

@ -21,7 +21,7 @@ public abstract class HttpRequest implements HttpMessage, HttpMessage.MessageTra
/**
* Returns the entity of this request.
*/
public abstract HttpEntityRegular entity();
public abstract RequestEntity entity();
/**
* Returns a copy of this instance with a new method.
@ -38,6 +38,11 @@ public abstract class HttpRequest implements HttpMessage, HttpMessage.MessageTra
*/
public abstract HttpRequest withUri(String path);
/**
* Returns a copy of this instance with a new entity.
*/
public abstract HttpRequest withEntity(RequestEntity entity);
/**
* Returns a default request to be changed using the `withX` methods.
*/

View file

@ -13,6 +13,11 @@ public abstract class HttpResponse implements HttpMessage, HttpMessage.MessageTr
*/
public abstract StatusCode status();
/**
* Returns the entity of this request.
*/
public abstract ResponseEntity entity();
/**
* Returns a copy of this instance with a new status-code.
*/
@ -23,6 +28,11 @@ public abstract class HttpResponse implements HttpMessage, HttpMessage.MessageTr
*/
public abstract HttpResponse withStatus(int statusCode);
/**
* Returns a copy of this instance with a new entity.
*/
public abstract HttpResponse withEntity(ResponseEntity entity);
/**
* Returns a default response to be changed using the `withX` methods.
*/

View file

@ -0,0 +1,8 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
/** Marker-interface for entity types that can be used in a request */
public interface RequestEntity extends ResponseEntity {}

View file

@ -0,0 +1,8 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
/** Marker-interface for entity types that can be used in a response */
public interface ResponseEntity extends HttpEntity {}

View file

@ -0,0 +1,8 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.model.japi;
/** Marker-interface for entity types that can be used in any context */
public interface UniversalEntity extends RequestEntity, ResponseEntity, BodyPartEntity {}

View file

@ -143,7 +143,7 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
def parseEntity(headers: List[HttpHeader], contentType: ContentType,
emitPartChunk: (List[HttpHeader], ContentType, ByteString) Unit = {
(headers, ct, bytes)
emit(BodyPartStart(headers, entityParts HttpEntity.CloseDelimited(ct,
emit(BodyPartStart(headers, entityParts HttpEntity.IndefiniteLength(ct,
Flow(entityParts).collect { case EntityPart(data) data }.toPublisher())))
emit(bytes)
},
@ -217,7 +217,7 @@ private[http] object BodyPartParser {
val boundaryCharNoSpace = CharPredicate.Digit ++ CharPredicate.Alpha ++ "'()+_,-./:=?"
sealed trait Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Publisher[Output] HttpEntity) extends Output
final case class BodyPartStart(headers: List[HttpHeader], createEntity: Publisher[Output] BodyPartEntity) extends Output
final case class EntityPart(data: ByteString) extends Output
final case class ParseError(info: ErrorInfo) extends Output

View file

@ -231,19 +231,19 @@ private[http] abstract class HttpMessageParser[Output >: ParserOutput.MessageOut
case None ContentTypes.`application/octet-stream`
}
def emptyEntity(cth: Option[`Content-Type`])(entityParts: Any): HttpEntity.Regular =
def emptyEntity(cth: Option[`Content-Type`])(entityParts: Any): UniversalEntity =
if (cth.isDefined) HttpEntity.empty(cth.get.contentType) else HttpEntity.Empty
def strictEntity(cth: Option[`Content-Type`], input: ByteString, bodyStart: Int,
contentLength: Int)(entityParts: Any): HttpEntity.Regular =
contentLength: Int)(entityParts: Any): UniversalEntity =
HttpEntity.Strict(contentType(cth), input.slice(bodyStart, bodyStart + contentLength))
def defaultEntity(cth: Option[`Content-Type`], contentLength: Long)(entityParts: Publisher[_ <: ParserOutput])(implicit fm: FlowMaterializer): HttpEntity.Regular = {
def defaultEntity(cth: Option[`Content-Type`], contentLength: Long)(entityParts: Publisher[_ <: ParserOutput])(implicit fm: FlowMaterializer): UniversalEntity = {
val data = Flow(entityParts).collect { case ParserOutput.EntityPart(bytes) bytes }.toPublisher()
HttpEntity.Default(contentType(cth), contentLength, data)
}
def chunkedEntity(cth: Option[`Content-Type`])(entityChunks: Publisher[_ <: ParserOutput])(implicit fm: FlowMaterializer): HttpEntity.Regular = {
def chunkedEntity(cth: Option[`Content-Type`])(entityChunks: Publisher[_ <: ParserOutput])(implicit fm: FlowMaterializer): RequestEntity with ResponseEntity = {
val chunks = Flow(entityChunks).collect { case ParserOutput.EntityChunk(chunk) chunk }.toPublisher()
HttpEntity.Chunked(contentType(cth), chunks)
}

View file

@ -108,7 +108,7 @@ private[http] class HttpRequestParser(_settings: ParserSettings,
clh: Option[`Content-Length`], cth: Option[`Content-Type`], teh: Option[`Transfer-Encoding`],
hostHeaderPresent: Boolean, closeAfterResponseCompletion: Boolean): StateResult =
if (hostHeaderPresent || protocol == HttpProtocols.`HTTP/1.0`) {
def emitRequestStart(createEntity: Publisher[ParserOutput.RequestOutput] HttpEntity.Regular) =
def emitRequestStart(createEntity: Publisher[ParserOutput.RequestOutput] RequestEntity) =
emit(ParserOutput.RequestStart(method, uri, protocol, headers, createEntity, closeAfterResponseCompletion))
teh match {

View file

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

View file

@ -27,14 +27,14 @@ private[http] object ParserOutput {
uri: Uri,
protocol: HttpProtocol,
headers: List[HttpHeader],
createEntity: Publisher[RequestOutput] HttpEntity.Regular,
createEntity: Publisher[RequestOutput] RequestEntity,
closeAfterResponseCompletion: Boolean) extends MessageStart with RequestOutput
final case class ResponseStart(
statusCode: StatusCode,
protocol: HttpProtocol,
headers: List[HttpHeader],
createEntity: Publisher[ResponseOutput] HttpEntity,
createEntity: Publisher[ResponseOutput] ResponseEntity,
closeAfterResponseCompletion: Boolean) extends MessageStart with ResponseOutput
case object MessageEnd extends MessageOutput

View file

@ -67,13 +67,10 @@ private[http] class BodyPartRenderer(boundary: String,
def completePartRendering(): List[Publisher[ChunkStreamPart]] =
bodyPart.entity match {
case x if x.isKnownEmpty chunkStream(r.get)
case Strict(_, data) chunkStream((r ~~ data).get)
case Default(_, _, data) bodyPartChunks(data)
case CloseDelimited(_, data) bodyPartChunks(data)
case Chunked(_, chunks)
val entityChunks = Flow(chunks).filter(!_.isLastChunk).toPublisher()
Flow(Chunk(r.get) :: Nil).concat(entityChunks).toPublisher() :: Nil
case x if x.isKnownEmpty chunkStream(r.get)
case Strict(_, data) chunkStream((r ~~ data).get)
case Default(_, _, data) bodyPartChunks(data)
case IndefiniteLength(_, data) bodyPartChunks(data)
}
renderBoundary()

View file

@ -118,7 +118,7 @@ private[http] class HttpResponseRendererFactory(serverHeader: Option[headers.Ser
def byteStrings(entityBytes: Publisher[ByteString]): List[Publisher[ByteString]] =
renderByteStrings(r, entityBytes, skipEntity = noEntity)
def completeResponseRendering(entity: HttpEntity): List[Publisher[ByteString]] =
def completeResponseRendering(entity: ResponseEntity): List[Publisher[ByteString]] =
entity match {
case HttpEntity.Strict(_, data)
renderHeaders(headers.toList)

View file

@ -72,9 +72,26 @@ sealed trait HttpEntity extends japi.HttpEntity {
// default implementations, should be overridden
def isCloseDelimited: Boolean = false
def isIndefiniteLength: Boolean = false
def isDefault: Boolean = false
def isChunked: Boolean = false
def isRegular: Boolean = false
}
/* An entity that can be used for body parts */
sealed trait BodyPartEntity extends HttpEntity with japi.BodyPartEntity {
def withContentType(contentType: ContentType): BodyPartEntity
}
/* An entity that can be used for requests */
sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity {
def withContentType(contentType: ContentType): RequestEntity
}
/* An entity that can be used for responses */
sealed trait ResponseEntity extends HttpEntity with japi.ResponseEntity {
def withContentType(contentType: ContentType): ResponseEntity
}
/* An entity that can be used for requests, responses, and body parts */
sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity {
def withContentType(contentType: ContentType): UniversalEntity
}
object HttpEntity {
@ -87,10 +104,10 @@ 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: Publisher[ByteString]): Regular =
def apply(contentType: ContentType, contentLength: Long, data: Publisher[ByteString]): UniversalEntity =
if (contentLength == 0) empty(contentType) else Default(contentType, contentLength, data)
def apply(contentType: ContentType, file: File): Regular = {
def apply(contentType: ContentType, file: File): UniversalEntity = {
val fileLength = file.length
if (fileLength > 0) Default(contentType, fileLength, ???) // FIXME: attach from-file-Publisher
else empty(contentType)
@ -102,23 +119,15 @@ object HttpEntity {
if (contentType == Empty.contentType) Empty
else Strict(contentType, data = ByteString.empty)
/**
* An HttpEntity that is "well-behaved" according to the HTTP/1.1 spec as that
* it is either chunked or defines a content-length that is known a-priori.
* Close-delimited entities are not `Regular` as they exists primarily for backwards compatibility with HTTP/1.0.
*/
sealed trait Regular extends japi.HttpEntityRegular with HttpEntity {
def withContentType(contentType: ContentType): HttpEntity.Regular
override def isRegular: Boolean = true
}
// TODO: re-establish serializability
// TODO: equal/hashcode ?
/**
* 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 Regular {
final case class Strict(contentType: ContentType, data: ByteString)
extends japi.HttpEntityStrict with UniversalEntity {
def isKnownEmpty: Boolean = data.isEmpty
def dataBytes(implicit fm: FlowMaterializer): Publisher[ByteString] = SynchronousPublisherFromIterable(data :: Nil)
@ -135,7 +144,9 @@ object HttpEntity {
*/
final case class Default(contentType: ContentType,
contentLength: Long,
data: Publisher[ByteString]) extends japi.HttpEntityDefault with Regular {
data: Publisher[ByteString])
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
@ -147,24 +158,51 @@ object HttpEntity {
}
/**
* The model for the entity of an HTTP response that is terminated by the server closing the connection.
* The content-length of such responses is unknown at the time the response headers have been received.
* Note that this type of HttpEntity cannot be used for HttpRequests!
* Supertype of CloseDelimited and IndefiniteLength.
*
* INTERNAL API
*/
final case class CloseDelimited(contentType: ContentType, data: Publisher[ByteString]) extends japi.HttpEntityCloseDelimited with HttpEntity {
private[http] sealed trait WithoutKnownLength extends HttpEntity {
def contentType: ContentType
def data: Publisher[ByteString]
def isKnownEmpty = data eq EmptyPublisher
override def isCloseDelimited: Boolean = true
def dataBytes(implicit fm: FlowMaterializer): Publisher[ByteString] = data
}
/**
* The model for the entity of an HTTP response that is terminated by the server closing the connection.
* 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: Publisher[ByteString])
extends japi.HttpEntityCloseDelimited with ResponseEntity with WithoutKnownLength {
type Self = CloseDelimited
override def isCloseDelimited: Boolean = true
def withContentType(contentType: ContentType): CloseDelimited =
if (contentType == this.contentType) this else copy(contentType = contentType)
}
/**
* 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: Publisher[ByteString])
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)
}
/**
* The model for the entity of a chunked HTTP message (with `Transfer-Encoding: chunked`).
*/
final case class Chunked(contentType: ContentType, chunks: Publisher[ChunkStreamPart]) extends japi.HttpEntityChunked with Regular {
final case class Chunked(contentType: ContentType, chunks: Publisher[ChunkStreamPart])
extends japi.HttpEntityChunked with MessageEntity {
def isKnownEmpty = chunks eq EmptyPublisher
override def isChunked: Boolean = true

View file

@ -25,7 +25,7 @@ sealed trait HttpMessage extends japi.HttpMessage {
def isResponse: Boolean
def headers: immutable.Seq[HttpHeader]
def entity: HttpEntity
def entity: ResponseEntity
def protocol: HttpProtocol
/** Returns a copy of this message with the list of headers set to the given ones. */
@ -44,20 +44,20 @@ sealed trait HttpMessage extends japi.HttpMessage {
}
/** Returns a copy of this message with the entity set to the given one. */
def withEntity(entity: japi.HttpEntity): Self
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): Deferrable[Self] =
entity.toStrict(timeout).map(this.withEntity)
/** Returns a copy of this message with the entity and headers set to the given ones. */
def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: HttpEntity): Self
def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: MessageEntity): Self
/** Returns a copy of this message with the list of headers transformed by the given function */
def mapHeaders(f: immutable.Seq[HttpHeader] immutable.Seq[HttpHeader]): Self = withHeaders(f(headers))
/** Returns a copy of this message with the entity transformed by the given function */
def mapEntity(f: HttpEntity HttpEntity): Self = withEntity(f(entity))
def mapEntity(f: HttpEntity MessageEntity): Self = withEntity(f(entity))
/**
* The content encoding as specified by the Content-Encoding header. If no Content-Encoding header is present the
@ -125,7 +125,7 @@ object HttpMessage {
final case class HttpRequest(method: HttpMethod = HttpMethods.GET,
uri: Uri = Uri./,
headers: immutable.Seq[HttpHeader] = Nil,
entity: HttpEntity.Regular = HttpEntity.Empty,
entity: RequestEntity = HttpEntity.Empty,
protocol: HttpProtocol = HttpProtocols.`HTTP/1.1`) extends japi.HttpRequest with HttpMessage {
require(!uri.isEmpty, "An HttpRequest must not have an empty Uri")
require(entity.isKnownEmpty || method.isEntityAccepted, "Requests with this method must have an empty entity")
@ -239,19 +239,11 @@ final case class HttpRequest(method: HttpMethod = HttpMethods.GET,
override def withHeaders(headers: immutable.Seq[HttpHeader]): HttpRequest =
if (headers eq this.headers) this else copy(headers = headers)
override def withEntity(entity: japi.HttpEntity): HttpRequest =
if (entity ne this.entity) entity match {
case x: HttpEntity.Regular copy(entity = x)
case _ throw new IllegalArgumentException("entity must be HttpEntity.Regular")
}
else this
override def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: RequestEntity): HttpRequest = copy(headers = headers, entity = entity)
override def withEntity(entity: japi.RequestEntity): HttpRequest = copy(entity = entity.asInstanceOf[RequestEntity])
override def withEntity(entity: MessageEntity): HttpRequest = copy(entity = entity)
override def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: HttpEntity): HttpRequest =
if ((headers ne this.headers) || (entity ne this.entity)) entity match {
case x: HttpEntity.Regular copy(headers = headers, entity = x)
case _ throw new IllegalArgumentException("entity must be HttpEntity.Regular")
}
else this
def mapEntity(f: RequestEntity RequestEntity): HttpRequest = withEntity(f(entity))
override def withMethod(method: akka.http.model.japi.HttpMethod): HttpRequest = copy(method = method.asInstanceOf[HttpMethod])
override def withProtocol(protocol: akka.http.model.japi.HttpProtocol): HttpRequest = copy(protocol = protocol.asInstanceOf[HttpProtocol])
@ -292,7 +284,7 @@ object HttpRequest {
*/
final case class HttpResponse(status: StatusCode = StatusCodes.OK,
headers: immutable.Seq[HttpHeader] = Nil,
entity: HttpEntity = HttpEntity.Empty,
entity: ResponseEntity = HttpEntity.Empty,
protocol: HttpProtocol = HttpProtocols.`HTTP/1.1`) extends japi.HttpResponse with HttpMessage {
type Self = HttpResponse
@ -302,12 +294,15 @@ final case class HttpResponse(status: StatusCode = StatusCodes.OK,
override def withHeaders(headers: immutable.Seq[HttpHeader]) =
if (headers eq this.headers) this else copy(headers = headers)
override def withEntity(entity: japi.HttpEntity) = if (entity eq this.entity) this else copy(entity = entity.asInstanceOf[HttpEntity])
override def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: HttpEntity) =
if ((headers eq this.headers) && (entity eq this.entity)) this else copy(headers = headers, entity = entity)
override def withProtocol(protocol: akka.http.model.japi.HttpProtocol): akka.http.model.japi.HttpResponse = copy(protocol = protocol.asInstanceOf[HttpProtocol])
override def withStatus(statusCode: Int): akka.http.model.japi.HttpResponse = copy(status = statusCode)
override def withStatus(statusCode: akka.http.model.japi.StatusCode): akka.http.model.japi.HttpResponse = copy(status = statusCode.asInstanceOf[StatusCode])
override def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: MessageEntity): HttpResponse = withHeadersAndEntity(headers, entity: ResponseEntity)
def withHeadersAndEntity(headers: immutable.Seq[HttpHeader], entity: ResponseEntity): HttpResponse = copy(headers = headers, entity = entity)
override def withEntity(entity: japi.ResponseEntity): HttpResponse = copy(entity = entity.asInstanceOf[ResponseEntity])
override def withEntity(entity: MessageEntity): HttpResponse = copy(entity = entity)
override def withEntity(entity: japi.RequestEntity): HttpResponse = withEntity(entity: japi.ResponseEntity)
def mapEntity(f: ResponseEntity ResponseEntity): HttpResponse = withEntity(f(entity))
}

View file

@ -85,16 +85,16 @@ object MultipartFormData {
}
}
final case class FormFile(name: Option[String], entity: HttpEntity)
final case class FormFile(name: Option[String], entity: BodyPartEntity)
object FormFile {
def apply(name: String, entity: HttpEntity): FormFile = apply(Some(name), entity)
def apply(name: String, entity: BodyPartEntity): FormFile = apply(Some(name), entity)
}
/**
* Model for one part of a multipart message.
*/
final case class BodyPart(entity: HttpEntity, headers: immutable.Seq[HttpHeader] = Nil) {
final case class BodyPart(entity: BodyPartEntity, headers: immutable.Seq[HttpHeader] = Nil) {
val name: Option[String] = dispositionParameterValue("name")
def filename: Option[String] = dispositionParameterValue("filename")
@ -126,8 +126,8 @@ object BodyPart {
case None apply(formFile.entity, fieldName)
}
def apply(entity: HttpEntity, fieldName: String): BodyPart = apply(entity, fieldName, Map.empty[String, String])
def apply(entity: HttpEntity, fieldName: String, params: Map[String, String]): BodyPart =
def apply(entity: BodyPartEntity, fieldName: String): BodyPart = apply(entity, fieldName, Map.empty[String, String])
def apply(entity: BodyPartEntity, fieldName: String, params: Map[String, String]): BodyPart =
BodyPart(entity, immutable.Seq(`Content-Disposition`(ContentDispositionTypes.`form-data`, params.updated("name", fieldName))))
}
@ -144,7 +144,7 @@ object BodyPart {
* }}}
*/
object NamedBodyPart {
def unapply(part: BodyPart): Option[(String, HttpEntity, immutable.Seq[HttpHeader])] =
def unapply(part: BodyPart): Option[(String, BodyPartEntity, immutable.Seq[HttpHeader])] =
part.name.map(name (name, part.entity, part.headers))
}
@ -162,6 +162,6 @@ object NamedBodyPart {
* }}}
*/
object FileBodyPart {
def unapply(part: BodyPart): Option[(String, String, HttpEntity, immutable.Seq[HttpHeader])] =
def unapply(part: BodyPart): Option[(String, String, BodyPartEntity, immutable.Seq[HttpHeader])] =
part.filename.map(filename (part.name.getOrElse(""), filename, part.entity, part.headers))
}

View file

@ -110,7 +110,6 @@ object JavaMapping {
implicit object HttpCharset extends Inherited[HttpCharset, model.HttpCharset]
implicit object HttpCharsetRange extends Inherited[HttpCharsetRange, model.HttpCharsetRange]
implicit object HttpEntity extends Inherited[HttpEntity, model.HttpEntity]
implicit object HttpEntityRegular extends Inherited[HttpEntityRegular, model.HttpEntity.Regular]
implicit object HttpHeader extends Inherited[HttpHeader, model.HttpHeader]
implicit object HttpMethod extends Inherited[HttpMethod, model.HttpMethod]
implicit object HttpProtocol extends Inherited[HttpProtocol, model.HttpProtocol]

View file

@ -0,0 +1,10 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http
package object model {
/** An entity that can be used for every HttpMessage, i.e. for requests and responses. */
type MessageEntity = RequestEntity
}

View file

@ -377,7 +377,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
private def newParser = new HttpRequestParser(ParserSettings(system), false)()
private def compactEntity(entity: HttpEntity): Deferrable[HttpEntity] =
private def compactEntity(entity: RequestEntity): Deferrable[RequestEntity] =
entity match {
case x: Chunked compactEntityChunks(x.chunks).map(compacted x.copy(chunks = compacted))
case _ entity.toStrict(250.millis)

View file

@ -239,7 +239,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
parser
}
private def compactEntity(entity: HttpEntity): Deferrable[HttpEntity] =
private def compactEntity(entity: ResponseEntity): Deferrable[ResponseEntity] =
entity match {
case x: HttpEntity.Chunked compactEntityChunks(x.chunks).map(compacted x.copy(chunks = compacted))
case _ entity.toStrict(250.millis)

View file

@ -39,7 +39,7 @@ trait RouteTestResultComponent {
def response: HttpResponse = rawResponse.copy(entity = entity)
/** Returns a "fresh" entity with a "fresh" unconsumed byte- or chunk stream (if not strict) */
def entity: HttpEntity = entityRecreator()
def entity: ResponseEntity = entityRecreator()
def chunks: immutable.Seq[ChunkStreamPart] =
entity match {
@ -76,7 +76,7 @@ trait RouteTestResultComponent {
this
}
private[this] lazy val entityRecreator: () HttpEntity =
private[this] lazy val entityRecreator: () ResponseEntity =
rawResponse.entity match {
case s: HttpEntity.Strict () s

View file

@ -42,6 +42,7 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
"The GenericMarshallers." - {
"optionMarshaller should enable marshalling of Option[T]" in {
marshal(Some("Ha“llo")) shouldEqual HttpEntity("Ha“llo")
marshal(None: Option[String]) shouldEqual HttpEntity.Empty
}

View file

@ -32,7 +32,7 @@ trait RequestBuilding extends TransformerPipelineSupport {
def apply[T](uri: String, content: Option[T])(implicit m: ToEntityMarshallers[T], ec: ExecutionContext): HttpRequest =
apply(Uri(uri), content)
def apply(uri: String, entity: HttpEntity.Regular): HttpRequest =
def apply(uri: String, entity: RequestEntity): HttpRequest =
apply(Uri(uri), entity)
def apply(uri: Uri): HttpRequest =
@ -45,11 +45,11 @@ trait RequestBuilding extends TransformerPipelineSupport {
content match {
case None apply(uri, HttpEntity.Empty)
case Some(value)
val entity = Marshal(value).to[HttpEntity.Regular].await(timeout.duration)
val entity = Marshal(value).to[RequestEntity].await(timeout.duration)
apply(uri, entity)
}
def apply(uri: Uri, entity: HttpEntity.Regular): HttpRequest =
def apply(uri: Uri, entity: RequestEntity): HttpRequest =
HttpRequest(method, uri, Nil, entity)
}

View file

@ -7,10 +7,10 @@ package akka.http.marshalling
import scala.collection.immutable
import akka.http.model._
class EmptyValue[T] private (val emptyValue: T)
class EmptyValue[+T] private (val emptyValue: T)
object EmptyValue {
implicit val emptyEntity = new EmptyValue[HttpEntity.Regular](HttpEntity.Empty)
implicit val emptyHeadersAndEntity = new EmptyValue[(immutable.Seq[HttpHeader], HttpEntity.Regular)](Nil -> HttpEntity.Empty)
implicit def emptyEntity = new EmptyValue[UniversalEntity](HttpEntity.Empty)
implicit val emptyHeadersAndEntity = new EmptyValue[(immutable.Seq[HttpHeader], UniversalEntity)](Nil -> HttpEntity.Empty)
implicit val emptyResponse = new EmptyValue[HttpResponse](HttpResponse(entity = emptyEntity.emptyValue))
}

View file

@ -11,7 +11,7 @@ import akka.http.util.Deferrable
import akka.http.model._
import MediaTypes._
case class Marshallers[-A, B](marshallers: immutable.Seq[Marshaller[A, B]]) {
case class Marshallers[-A, +B](marshallers: immutable.Seq[Marshaller[A, B]]) {
require(marshallers.nonEmpty, "marshallers must be non-empty")
def map[C](f: B C)(implicit ec: ExecutionContext): Marshallers[A, C] =
Marshallers(marshallers map (_ map f))
@ -30,18 +30,15 @@ object Marshallers extends SingleMarshallerMarshallers {
Marshallers(`text/xml`, `application/xml`, `text/html`, `application/xhtml+xml`)(PredefinedToEntityMarshallers.nodeSeqMarshaller)
}
implicit def entity2response[T](implicit m: Marshallers[T, HttpEntity], ec: ExecutionContext): Marshallers[T, HttpResponse] =
implicit def entity2response[T](implicit m: Marshallers[T, ResponseEntity], ec: ExecutionContext): Marshallers[T, HttpResponse] =
m map (entity HttpResponse(entity = entity))
implicit def regularEntity2entity[T](implicit m: Marshallers[T, HttpEntity.Regular], ec: ExecutionContext): Marshallers[T, HttpEntity] =
m map identity
}
sealed abstract class SingleMarshallerMarshallers {
implicit def singleMarshallerMarshallers[A, B](implicit m: Marshaller[A, B]): Marshallers[A, B] = Marshallers(m)
}
sealed trait Marshaller[-A, B] { outer
sealed trait Marshaller[-A, +B] { outer
def apply(value: A): Deferrable[Marshalling[B]]
def map[C](f: B C)(implicit ec: ExecutionContext): Marshaller[A, C] =
@ -51,7 +48,7 @@ sealed trait Marshaller[-A, B] { outer ⇒
* Reuses this Marshaller's logic to produce a new Marshaller from another type `C` which overrides
* the produced media-type with another one.
*/
def wrap[C](mediaType: MediaType)(f: C A)(implicit ec: ExecutionContext, mto: MediaTypeOverrider[B]): Marshaller[C, B] =
def wrap[C, D >: B](mediaType: MediaType)(f: C A)(implicit ec: ExecutionContext, mto: MediaTypeOverrider[D]): Marshaller[C, D] =
Marshaller { value
import Marshalling._
outer(f(value)) map {

View file

@ -11,28 +11,20 @@ sealed trait MediaTypeOverrider[T] {
def apply(value: T, mediaType: MediaType): T
}
object MediaTypeOverrider {
implicit val forRegularEntity = new MediaTypeOverrider[HttpEntity.Regular] {
def apply(value: HttpEntity.Regular, mediaType: MediaType) =
value.withContentType(value.contentType withMediaType mediaType)
implicit def forEntity[T <: HttpEntity]: MediaTypeOverrider[T] = new MediaTypeOverrider[T] {
def apply(value: T, mediaType: MediaType) =
value.withContentType(value.contentType withMediaType mediaType).asInstanceOf[T] // can't be expressed in types
}
implicit val forEntity = new MediaTypeOverrider[HttpEntity] {
def apply(value: HttpEntity, mediaType: MediaType) =
value.withContentType(value.contentType withMediaType mediaType)
}
implicit val forHeadersAndRegularEntity = new MediaTypeOverrider[(immutable.Seq[HttpHeader], HttpEntity.Regular)] {
def apply(value: (immutable.Seq[HttpHeader], HttpEntity.Regular), mediaType: MediaType) =
value._1 -> value._2.withContentType(value._2.contentType withMediaType mediaType)
}
implicit val forHeadersAndEntity = new MediaTypeOverrider[(immutable.Seq[HttpHeader], HttpEntity)] {
def apply(value: (immutable.Seq[HttpHeader], HttpEntity), mediaType: MediaType) =
value._1 -> value._2.withContentType(value._2.contentType withMediaType mediaType)
implicit def forHeadersAndEntity[T <: HttpEntity] = new MediaTypeOverrider[(immutable.Seq[HttpHeader], T)] {
def apply(value: (immutable.Seq[HttpHeader], T), mediaType: MediaType) =
value._1 -> value._2.withContentType(value._2.contentType withMediaType mediaType).asInstanceOf[T]
}
implicit val forResponse = new MediaTypeOverrider[HttpResponse] {
def apply(value: HttpResponse, mediaType: MediaType) =
value.mapEntity(forEntity(_, mediaType))
value.mapEntity(forEntity(_: ResponseEntity, mediaType))
}
implicit val forRequest = new MediaTypeOverrider[HttpRequest] {
def apply(value: HttpRequest, mediaType: MediaType) =
value.mapEntity(forEntity(_, mediaType))
value.mapEntity(forEntity(_: RequestEntity, mediaType))
}
}

View file

@ -50,7 +50,7 @@ trait MultipartMarshallers {
mcm(MultipartContent(value.parts)) map {
case Marshalling.WithOpenCharset(mt, marshal)
val mediaType = `multipart/form-data` withBoundary mt.params("boundary")
Marshalling.WithOpenCharset(mediaType, cs MediaTypeOverrider.forRegularEntity(marshal(cs), mediaType))
Marshalling.WithOpenCharset(mediaType, cs MediaTypeOverrider.forEntity(marshal(cs), mediaType))
case x throw new IllegalStateException("ToRegularEntityMarshaller[MultipartContent] is expected to produce " +
"a Marshalling.WithOpenCharset, not a " + x)
}

View file

@ -61,7 +61,7 @@ trait PredefinedToEntityMarshallers extends MultipartMarshallers {
HttpEntity(ContentType(`application/x-www-form-urlencoded`, charset), string)
}
implicit val HttpEntityMarshaller: ToEntityMarshaller[HttpEntity.Regular] = Marshaller { value
implicit val HttpEntityMarshaller: ToEntityMarshaller[MessageEntity] = Marshaller { value
// since we don't want to recode we simply ignore the charset determined by content negotiation here
Deferrable(Marshalling.WithOpenCharset(value.contentType.mediaType, _ value))
}

View file

@ -8,12 +8,12 @@ import scala.collection.immutable
import akka.http.model._
package object marshalling {
type ToEntityMarshaller[T] = Marshaller[T, HttpEntity.Regular]
type ToHeadersAndEntityMarshaller[T] = Marshaller[T, (immutable.Seq[HttpHeader], HttpEntity.Regular)]
type ToEntityMarshaller[T] = Marshaller[T, MessageEntity]
type ToHeadersAndEntityMarshaller[T] = Marshaller[T, (immutable.Seq[HttpHeader], MessageEntity)]
type ToResponseMarshaller[T] = Marshaller[T, HttpResponse]
type ToRequestMarshaller[T] = Marshaller[T, HttpRequest]
type ToEntityMarshallers[T] = Marshallers[T, HttpEntity.Regular]
type ToEntityMarshallers[T] = Marshallers[T, MessageEntity]
type ToResponseMarshallers[T] = Marshallers[T, HttpResponse]
type ToRequestMarshallers[T] = Marshallers[T, HttpRequest]
}

View file

@ -106,7 +106,7 @@ trait RequestContext {
/**
* Returns a copy of this context with the given response transformation function chained into the response chain.
*/
def withHttpResponseEntityMapped(f: HttpEntity HttpEntity): RequestContext
def withHttpResponseEntityMapped(f: ResponseEntity ResponseEntity): RequestContext
/**
* Returns a copy of this context with the given response transformation function chained into the response chain.

View file

@ -72,7 +72,7 @@ private[http] class RequestContextImpl(
case RouteResult.Complete(response) RouteResult.complete(f(response))
}
override def withHttpResponseEntityMapped(f: HttpEntity HttpEntity): RequestContext =
override def withHttpResponseEntityMapped(f: ResponseEntity ResponseEntity): RequestContext =
withHttpResponseMapped(_ mapEntity f)
override def withHttpResponseHeadersMapped(f: immutable.Seq[HttpHeader] immutable.Seq[HttpHeader]): RequestContext =

View file

@ -33,7 +33,7 @@ trait BasicDirectives {
def mapHttpResponse(f: HttpResponse HttpResponse): Directive0 =
mapRequestContext(_ withHttpResponseMapped f)
def mapHttpResponseEntity(f: HttpEntity HttpEntity): Directive0 =
def mapHttpResponseEntity(f: ResponseEntity ResponseEntity): Directive0 =
mapRequestContext(_ withHttpResponseEntityMapped f)
def mapHttpResponseHeaders(f: immutable.Seq[HttpHeader] immutable.Seq[HttpHeader]): Directive0 =