Merge pull request #15984 from spray/w/15910-encoding-infrastructure
+htp #15910 import coding infrastructure from spray
This commit is contained in:
commit
be581dffc1
17 changed files with 1073 additions and 3 deletions
|
|
@ -14,11 +14,14 @@ import scala.concurrent.{ Future, ExecutionContext }
|
|||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.collection.immutable
|
||||
import akka.util.ByteString
|
||||
import akka.stream.{ TimerTransformer, FlowMaterializer }
|
||||
import akka.stream.{ Transformer, TimerTransformer, FlowMaterializer }
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.impl.{ EmptyPublisher, SynchronousPublisherFromIterable }
|
||||
import akka.http.util._
|
||||
import japi.JavaMapping.Implicits._
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* Models the entity (aka "body" or "content) of an HTTP message.
|
||||
*/
|
||||
|
|
@ -63,6 +66,16 @@ sealed trait HttpEntity extends japi.HttpEntity {
|
|||
Flow(dataBytes).timerTransform("toStrict", transformer).toFuture()
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a copy of the given entity with the ByteString chunks of this entity transformed by the given transformer.
|
||||
* For a `Chunked` entity, the chunks will be transformed one by one keeping the chunk metadata (but may introduce an
|
||||
* extra chunk before the `LastChunk` if `transformer.onTermination` returns additional data).
|
||||
*
|
||||
* 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: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): HttpEntity
|
||||
|
||||
/**
|
||||
* Creates a copy of this HttpEntity with the `contentType` overridden with the given one.
|
||||
*/
|
||||
|
|
@ -85,10 +98,14 @@ sealed trait BodyPartEntity extends HttpEntity with japi.BodyPartEntity {
|
|||
/* An entity that can be used for requests */
|
||||
sealed trait RequestEntity extends HttpEntity with japi.RequestEntity with ResponseEntity {
|
||||
def withContentType(contentType: ContentType): RequestEntity
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): 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: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): ResponseEntity
|
||||
}
|
||||
/* An entity that can be used for requests, responses, and body parts */
|
||||
sealed trait UniversalEntity extends japi.UniversalEntity with MessageEntity with BodyPartEntity {
|
||||
|
|
@ -136,6 +153,17 @@ object HttpEntity {
|
|||
override def toStrict(timeout: FiniteDuration)(implicit ec: ExecutionContext, fm: FlowMaterializer) =
|
||||
FastFuture.successful(this)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): MessageEntity = {
|
||||
try {
|
||||
val t = transformer()
|
||||
val newData = (t.onNext(data) ++ t.onTermination(None)).join
|
||||
copy(data = newData)
|
||||
} catch {
|
||||
case NonFatal(ex) ⇒
|
||||
Chunked(contentType, StreamUtils.failedPublisher(ex))
|
||||
}
|
||||
}
|
||||
|
||||
def withContentType(contentType: ContentType): Strict =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
}
|
||||
|
|
@ -154,6 +182,12 @@ object HttpEntity {
|
|||
|
||||
def dataBytes(implicit fm: FlowMaterializer): Publisher[ByteString] = data
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): Chunked = {
|
||||
val chunks = Flow(data).transform("transformDataBytes-Default", () ⇒ transformer().map(Chunk(_): ChunkStreamPart)).toPublisher()
|
||||
|
||||
HttpEntity.Chunked(contentType, chunks)
|
||||
}
|
||||
|
||||
def withContentType(contentType: ContentType): Default =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
}
|
||||
|
|
@ -184,6 +218,10 @@ object HttpEntity {
|
|||
override def isCloseDelimited: Boolean = true
|
||||
def withContentType(contentType: ContentType): CloseDelimited =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): CloseDelimited =
|
||||
HttpEntity.CloseDelimited(contentType,
|
||||
Flow(data).transform("transformDataBytes-CloseDelimited", transformer).toPublisher())
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -196,6 +234,10 @@ object HttpEntity {
|
|||
override def isIndefiniteLength: Boolean = true
|
||||
def withContentType(contentType: ContentType): IndefiniteLength =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): IndefiniteLength =
|
||||
HttpEntity.IndefiniteLength(contentType,
|
||||
Flow(data).transform("transformDataBytes-IndefiniteLength", transformer).toPublisher())
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -210,6 +252,35 @@ object HttpEntity {
|
|||
def dataBytes(implicit fm: FlowMaterializer): Publisher[ByteString] =
|
||||
Flow(chunks).map(_.data).filter(_.nonEmpty).toPublisher()
|
||||
|
||||
override def transformDataBytes(transformer: () ⇒ Transformer[ByteString, ByteString])(implicit materializer: FlowMaterializer): Chunked = {
|
||||
val newChunks =
|
||||
Flow(chunks).transform("transformDataBytes-Chunked", () ⇒ new Transformer[ChunkStreamPart, ChunkStreamPart] {
|
||||
val byteTransformer = transformer()
|
||||
var sentLastChunk = false
|
||||
|
||||
override def isComplete: Boolean = byteTransformer.isComplete
|
||||
|
||||
def onNext(element: ChunkStreamPart): immutable.Seq[ChunkStreamPart] = element match {
|
||||
case Chunk(data, ext) ⇒ Chunk(byteTransformer.onNext(data).join, ext) :: Nil
|
||||
case l: LastChunk ⇒
|
||||
sentLastChunk = true
|
||||
Chunk(byteTransformer.onTermination(None).join) :: l :: Nil
|
||||
}
|
||||
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ChunkStreamPart] = {
|
||||
val remaining =
|
||||
if (e.isEmpty && !sentLastChunk) byteTransformer.onTermination(None)
|
||||
else if (e.isDefined /* && sentLastChunk */ ) byteTransformer.onTermination(e)
|
||||
else Nil
|
||||
|
||||
if (remaining.nonEmpty) Chunk(remaining.join) :: Nil
|
||||
else Nil
|
||||
}
|
||||
}).toPublisher()
|
||||
|
||||
HttpEntity.Chunked(contentType, newChunks)
|
||||
}
|
||||
|
||||
def withContentType(contentType: ContentType): Chunked =
|
||||
if (contentType == this.contentType) this else copy(contentType = contentType)
|
||||
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import FastFuture._
|
|||
*/
|
||||
sealed trait HttpMessage extends japi.HttpMessage {
|
||||
type Self <: HttpMessage
|
||||
def self: Self
|
||||
|
||||
def isRequest: Boolean
|
||||
def isResponse: Boolean
|
||||
|
|
@ -134,6 +135,7 @@ final case class HttpRequest(method: HttpMethod = HttpMethods.GET,
|
|||
"HTTP/1.0 requests must not have a chunked entity")
|
||||
|
||||
type Self = HttpRequest
|
||||
def self = this
|
||||
|
||||
override def isRequest = true
|
||||
override def isResponse = false
|
||||
|
|
@ -288,6 +290,7 @@ final case class HttpResponse(status: StatusCode = StatusCodes.OK,
|
|||
entity: ResponseEntity = HttpEntity.Empty,
|
||||
protocol: HttpProtocol = HttpProtocols.`HTTP/1.1`) extends japi.HttpResponse with HttpMessage {
|
||||
type Self = HttpResponse
|
||||
def self = this
|
||||
|
||||
override def isRequest = false
|
||||
override def isResponse = true
|
||||
|
|
|
|||
|
|
@ -0,0 +1,14 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.http.util
|
||||
|
||||
import akka.util.ByteString
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] class EnhancedByteStringTraversableOnce(val byteStrings: TraversableOnce[ByteString]) extends AnyVal {
|
||||
def join: ByteString = byteStrings.foldLeft(ByteString.empty)(_ ++ _)
|
||||
}
|
||||
|
|
@ -0,0 +1,54 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.http.util
|
||||
|
||||
import akka.stream.impl.ErrorPublisher
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.{ FlowMaterializer, Transformer }
|
||||
import akka.util.ByteString
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[http] object StreamUtils {
|
||||
/**
|
||||
* Maps a transformer by strictly applying the given function to each output element.
|
||||
*/
|
||||
def mapTransformer[T, U, V](t: Transformer[T, U], f: U ⇒ V): Transformer[T, V] =
|
||||
new Transformer[T, V] {
|
||||
override def isComplete: Boolean = t.isComplete
|
||||
|
||||
def onNext(element: T): immutable.Seq[V] = t.onNext(element).map(f)
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[V] = t.onTermination(e).map(f)
|
||||
override def onError(cause: Throwable): Unit = t.onError(cause)
|
||||
override def cleanup(): Unit = t.cleanup()
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a transformer that will call `f` for each incoming ByteString and output its result. After the complete
|
||||
* input has been read it will call `finish` once to determine the final ByteString to post to the output.
|
||||
*/
|
||||
def byteStringTransformer(f: ByteString ⇒ ByteString, finish: () ⇒ ByteString): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
def onNext(element: ByteString): immutable.Seq[ByteString] = f(element) :: Nil
|
||||
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ByteString] =
|
||||
if (e.isEmpty) {
|
||||
val last = finish()
|
||||
if (last.nonEmpty) last :: Nil
|
||||
else Nil
|
||||
} else super.onTermination(e)
|
||||
}
|
||||
|
||||
def failedPublisher[T](ex: Throwable): Publisher[T] =
|
||||
ErrorPublisher(ex).asInstanceOf[Publisher[T]]
|
||||
}
|
||||
|
||||
private[http] class EnhancedTransformer[T, U](val t: Transformer[T, U]) extends AnyVal {
|
||||
def map[V](f: U ⇒ V): Transformer[T, V] = StreamUtils.mapTransformer(t, f)
|
||||
}
|
||||
|
|
@ -30,6 +30,10 @@ package object util {
|
|||
private[http] implicit def enhanceConfig(config: Config): EnhancedConfig = new EnhancedConfig(config)
|
||||
private[http] implicit def enhanceString_(s: String): EnhancedString = new EnhancedString(s)
|
||||
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 enhanceTransformer[T, U](transformer: Transformer[T, U]): EnhancedTransformer[T, U] =
|
||||
new EnhancedTransformer(transformer)
|
||||
|
||||
private[http] implicit class FlowWithHeadAndTail[T](val underlying: Flow[Publisher[T]]) extends AnyVal {
|
||||
def headAndTail(implicit fm: FlowMaterializer): Flow[(T, Publisher[T])] =
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.http.model
|
|||
import java.util.concurrent.TimeoutException
|
||||
import com.typesafe.config.{ ConfigFactory, Config }
|
||||
import org.reactivestreams.Publisher
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.{ Promise, Await }
|
||||
import scala.concurrent.duration._
|
||||
import org.scalatest.{ BeforeAndAfterAll, MustMatchers, FreeSpec }
|
||||
|
|
@ -14,10 +15,9 @@ import org.scalatest.matchers.Matcher
|
|||
import akka.util.ByteString
|
||||
import akka.actor.ActorSystem
|
||||
import akka.stream.scaladsl.Flow
|
||||
import akka.stream.FlowMaterializer
|
||||
import akka.stream.{ Transformer, FlowMaterializer }
|
||||
import akka.stream.impl.SynchronousPublisherFromIterable
|
||||
import akka.http.model.HttpEntity._
|
||||
import akka.http.util.FastFuture._
|
||||
|
||||
class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
||||
val tpe: ContentType = ContentTypes.`application/octet-stream`
|
||||
|
|
@ -81,6 +81,27 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
|||
}.getMessage must be("HttpEntity.toStrict timed out after 100 milliseconds while still waiting for outstanding data")
|
||||
}
|
||||
}
|
||||
"support transformDataBytes" - {
|
||||
"Strict" in {
|
||||
Strict(tpe, abc) must transformTo(Strict(tpe, doubleChars("abc") ++ trailer))
|
||||
}
|
||||
"Default" in {
|
||||
Default(tpe, 11, publisher(abc, de, fgh, ijk)) must
|
||||
transformTo(Strict(tpe, doubleChars("abcdefghijk") ++ trailer))
|
||||
}
|
||||
"CloseDelimited" in {
|
||||
CloseDelimited(tpe, publisher(abc, de, fgh, ijk)) must
|
||||
transformTo(Strict(tpe, doubleChars("abcdefghijk") ++ trailer))
|
||||
}
|
||||
"Chunked w/o LastChunk" in {
|
||||
Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk))) must
|
||||
transformTo(Strict(tpe, doubleChars("abcfghijk") ++ trailer))
|
||||
}
|
||||
"Chunked with LastChunk" in {
|
||||
Chunked(tpe, publisher(Chunk(abc), Chunk(fgh), Chunk(ijk), LastChunk)) must
|
||||
transformTo(Strict(tpe, doubleChars("abcfghijk") ++ trailer))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def publisher[T](elems: T*) = SynchronousPublisherFromIterable(elems.toList)
|
||||
|
|
@ -93,4 +114,23 @@ class HttpEntitySpec extends FreeSpec with MustMatchers with BeforeAndAfterAll {
|
|||
|
||||
def strictifyTo(strict: Strict): Matcher[HttpEntity] =
|
||||
equal(strict).matcher[Strict].compose(x ⇒ Await.result(x.toStrict(250.millis), 250.millis))
|
||||
|
||||
def transformTo(strict: Strict): Matcher[HttpEntity] =
|
||||
equal(strict).matcher[Strict].compose { x ⇒
|
||||
val transformed = x.transformDataBytes(duplicateBytesTransformer)
|
||||
Await.result(transformed.toStrict(250.millis), 250.millis)
|
||||
}
|
||||
|
||||
def duplicateBytesTransformer(): Transformer[ByteString, ByteString] =
|
||||
new Transformer[ByteString, ByteString] {
|
||||
def onNext(bs: ByteString): immutable.Seq[ByteString] =
|
||||
Vector(doubleChars(bs))
|
||||
|
||||
override def onTermination(e: Option[Throwable]): immutable.Seq[ByteString] =
|
||||
Vector(trailer)
|
||||
}
|
||||
|
||||
def trailer: ByteString = ByteString("--dup")
|
||||
def doubleChars(bs: ByteString): ByteString = ByteString(bs.flatMap(b ⇒ Seq(b, b)): _*)
|
||||
def doubleChars(str: String): ByteString = doubleChars(ByteString(str))
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue