diff --git a/akka-http-tests/src/test/scala/akka/http/marshalling/MarshallingSpec.scala b/akka-http-tests/src/test/scala/akka/http/marshalling/MarshallingSpec.scala
index 98d838c343..6f75f59271 100644
--- a/akka-http-tests/src/test/scala/akka/http/marshalling/MarshallingSpec.scala
+++ b/akka-http-tests/src/test/scala/akka/http/marshalling/MarshallingSpec.scala
@@ -10,6 +10,7 @@ import scala.concurrent.duration._
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
import akka.actor.ActorSystem
import akka.stream.FlowMaterializer
+import akka.stream.scaladsl.Source
import akka.http.util._
import akka.http.model._
import headers._
@@ -51,9 +52,9 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
}
"The MultipartMarshallers." - {
- "multipartContentMarshaller should correctly marshal multipart content with" - {
+ "multipartMarshaller should correctly marshal multipart content with" - {
"one empty part" in {
- marshal(MultipartContent(BodyPart(""))) shouldEqual HttpEntity(
+ marshal(Multipart.General(`multipart/mixed`, Multipart.General.BodyPart.Strict(""))) shouldEqual HttpEntity(
contentType = ContentType(`multipart/mixed` withBoundary randomBoundary),
string = s"""--$randomBoundary
|Content-Type: text/plain; charset=UTF-8
@@ -62,11 +63,11 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
|--$randomBoundary--""".stripMarginWithNewline("\r\n"))
}
"one non-empty part" in {
- marshal(MultipartContent(BodyPart(
+ marshal(Multipart.General(`multipart/alternative`, Multipart.General.BodyPart.Strict(
entity = HttpEntity(ContentType(`text/plain`, `UTF-8`), "test@there.com"),
headers = `Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "email")) :: Nil))) shouldEqual
HttpEntity(
- contentType = ContentType(`multipart/mixed` withBoundary randomBoundary),
+ contentType = ContentType(`multipart/alternative` withBoundary randomBoundary),
string = s"""--$randomBoundary
|Content-Type: text/plain; charset=UTF-8
|Content-Disposition: form-data; name=email
@@ -75,13 +76,13 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
|--$randomBoundary--""".stripMarginWithNewline("\r\n"))
}
"two different parts" in {
- marshal(MultipartContent(
- BodyPart(HttpEntity(ContentType(`text/plain`, Some(`US-ASCII`)), "first part, with a trailing linebreak\r\n")),
- BodyPart(
+ marshal(Multipart.General(`multipart/related`,
+ Multipart.General.BodyPart.Strict(HttpEntity(ContentType(`text/plain`, Some(`US-ASCII`)), "first part, with a trailing linebreak\r\n")),
+ Multipart.General.BodyPart.Strict(
HttpEntity(ContentType(`application/octet-stream`), "filecontent"),
RawHeader("Content-Transfer-Encoding", "binary") :: Nil))) shouldEqual
HttpEntity(
- contentType = ContentType(`multipart/mixed` withBoundary randomBoundary),
+ contentType = ContentType(`multipart/related` withBoundary randomBoundary),
string = s"""--$randomBoundary
|Content-Type: text/plain; charset=US-ASCII
|
@@ -98,9 +99,9 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
"multipartFormDataMarshaller should correctly marshal 'multipart/form-data' content with" - {
"two fields" in {
- marshal(MultipartFormData(ListMap(
- "surname" -> BodyPart("Mike"),
- "age" -> BodyPart(marshal(42))))) shouldEqual
+ marshal(Multipart.FormData(ListMap(
+ "surname" -> HttpEntity("Mike"),
+ "age" -> marshal(42)))) shouldEqual
HttpEntity(
contentType = ContentType(`multipart/form-data` withBoundary randomBoundary),
string = s"""--$randomBoundary
@@ -117,32 +118,26 @@ class MarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll with
}
"two fields having a custom `Content-Disposition`" in {
- marshal(MultipartFormData(
- BodyPart(
- HttpEntity(`text/csv`, "name,age\r\n\"John Doe\",20\r\n"),
- List(`Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "attachment[0]", "filename" -> "attachment.csv")))),
- BodyPart(
- HttpEntity("name,age\r\n\"John Doe\",20\r\n".getBytes),
- List(
- `Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "attachment[1]", "filename" -> "attachment.csv")),
- RawHeader("Content-Transfer-Encoding", "binary"))))) shouldEqual
+ marshal(Multipart.FormData(Source(List(
+ Multipart.FormData.BodyPart("attachment[0]", HttpEntity(`text/csv`, "name,age\r\n\"John Doe\",20\r\n"),
+ Map("filename" -> "attachment.csv")),
+ Multipart.FormData.BodyPart("attachment[1]", HttpEntity("naice!".getBytes),
+ Map("filename" -> "attachment2.csv"), List(RawHeader("Content-Transfer-Encoding", "binary"))))))) shouldEqual
HttpEntity(
contentType = ContentType(`multipart/form-data` withBoundary randomBoundary),
string = s"""--$randomBoundary
|Content-Type: text/csv
- |Content-Disposition: form-data; name="attachment[0]"; filename=attachment.csv
+ |Content-Disposition: form-data; filename=attachment.csv; name="attachment[0]"
|
|name,age
|"John Doe",20
|
|--$randomBoundary
|Content-Type: application/octet-stream
- |Content-Disposition: form-data; name="attachment[1]"; filename=attachment.csv
+ |Content-Disposition: form-data; filename=attachment2.csv; name="attachment[1]"
|Content-Transfer-Encoding: binary
|
- |name,age
- |"John Doe",20
- |
+ |naice!
|--$randomBoundary--""".stripMarginWithNewline("\r\n"))
}
}
diff --git a/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala b/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala
index df07ae9cb6..190f95580f 100644
--- a/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala
+++ b/akka-http-tests/src/test/scala/akka/http/unmarshalling/UnmarshallingSpec.scala
@@ -4,6 +4,8 @@
package akka.http.unmarshalling
+import akka.util.ByteString
+
import scala.xml.NodeSeq
import scala.concurrent.duration._
import scala.concurrent.{ Future, Await }
@@ -37,25 +39,50 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
"The MultipartUnmarshallers." - {
- "multipartContentUnmarshaller should correctly unmarshal 'multipart/*' content with" - {
- "one empty part" in {
+ "multipartGeneralUnmarshaller should correctly unmarshal 'multipart/*' content with" - {
+ "an empty entity" in {
+ Unmarshal(HttpEntity(`multipart/mixed` withBoundary "XYZABC", ByteString.empty)).to[Multipart.General] should haveParts()
+ }
+ "an entity without initial boundary" in {
+ Unmarshal(HttpEntity(`multipart/mixed` withBoundary "XYZABC",
+ """this is
+ |just preamble text""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts()
+ }
+ "an empty part" in {
Unmarshal(HttpEntity(`multipart/mixed` withBoundary "XYZABC",
"""--XYZABC
- |--XYZABC--""".stripMargin)).to[MultipartContent] should haveParts()
+ |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(HttpEntity.empty(ContentTypes.`text/plain(UTF-8)`)))
}
- "with one part" in {
+ "two empty parts" in {
+ Unmarshal(HttpEntity(`multipart/mixed` withBoundary "XYZABC",
+ """--XYZABC
+ |--XYZABC
+ |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(HttpEntity.empty(ContentTypes.`text/plain(UTF-8)`)),
+ Multipart.General.BodyPart.Strict(HttpEntity.empty(ContentTypes.`text/plain(UTF-8)`)))
+ }
+ "a part without entity and missing header separation CRLF" in {
+ Unmarshal(HttpEntity(`multipart/mixed` withBoundary "XYZABC",
+ """--XYZABC
+ |Content-type: text/xml
+ |Age: 12
+ |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(HttpEntity.empty(MediaTypes.`text/xml`), List(RawHeader("Age", "12"))))
+ }
+ "one non-empty part" in {
Unmarshal(HttpEntity(`multipart/form-data` withBoundary "-",
"""---
|Content-type: text/plain; charset=UTF8
|content-disposition: form-data; name="email"
|
|test@there.com
- |-----""".stripMarginWithNewline("\r\n"))).to[MultipartContent] should haveParts(
- BodyPart(
+ |-----""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(
HttpEntity(ContentTypes.`text/plain(UTF-8)`, "test@there.com"),
List(`Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "email")))))
}
- "with two different parts" in {
+ "two different parts" in {
Unmarshal(HttpEntity(`multipart/mixed` withBoundary "12345",
"""--12345
|
@@ -66,38 +93,48 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|Content-Transfer-Encoding: binary
|
|filecontent
- |--12345--""".stripMarginWithNewline("\r\n"))).to[MultipartContent] should haveParts(
- BodyPart(HttpEntity(ContentTypes.`text/plain(UTF-8)`, "first part, with a trailing newline\r\n")),
- BodyPart(
+ |--12345--""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(HttpEntity(ContentTypes.`text/plain(UTF-8)`, "first part, with a trailing newline\r\n")),
+ Multipart.General.BodyPart.Strict(
HttpEntity(`application/octet-stream`, "filecontent"),
List(RawHeader("Content-Transfer-Encoding", "binary"))))
}
- "with illegal headers" in (
+ "illegal headers" in (
Unmarshal(HttpEntity(`multipart/form-data` withBoundary "XYZABC",
"""--XYZABC
|Date: unknown
|content-disposition: form-data; name=email
|
|test@there.com
- |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[MultipartContent] should haveParts(
- BodyPart(
+ |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[Multipart.General] should haveParts(
+ Multipart.General.BodyPart.Strict(
HttpEntity(ContentTypes.`text/plain(UTF-8)`, "test@there.com"),
List(`Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "email")),
RawHeader("date", "unknown")))))
}
- "multipartContentUnmarshaller should reject illegal multipart content" in {
- val mpc = Await.result(Unmarshal(HttpEntity(`multipart/form-data` withBoundary "-",
- """---
- |Content-type: text/plain; charset=UTF8
- |Content-type: application/json
- |content-disposition: form-data; name="email"
- |
- |test@there.com
- |-----""".stripMarginWithNewline("\r\n")))
- .to[MultipartContent], 1.second)
- Await.result(mpc.parts.runWith(Sink.future).failed, 1.second).getMessage shouldEqual
- "multipart part must not contain more than one Content-Type header"
+ "multipartGeneralUnmarshaller should reject illegal multipart content with" - {
+ "a stray boundary" in {
+ Await.result(Unmarshal(HttpEntity(`multipart/form-data` withBoundary "ABC",
+ """--ABC
+ |Content-type: text/plain; charset=UTF8
+ |--ABCContent-type: application/json
+ |content-disposition: form-data; name="email"
+ |-----""".stripMarginWithNewline("\r\n")))
+ .to[Multipart.General].failed, 1.second).getMessage shouldEqual "Illegal multipart boundary in message content"
+ }
+ "duplicate Content-Type header" in {
+ Await.result(Unmarshal(HttpEntity(`multipart/form-data` withBoundary "-",
+ """---
+ |Content-type: text/plain; charset=UTF8
+ |Content-type: application/json
+ |content-disposition: form-data; name="email"
+ |
+ |test@there.com
+ |-----""".stripMarginWithNewline("\r\n")))
+ .to[Multipart.General].failed, 1.second).getMessage shouldEqual
+ "multipart part must not contain more than one Content-Type header"
+ }
}
"multipartByteRangesUnmarshaller should correctly unmarshal multipart/byteranges content with two different parts" in {
@@ -112,9 +149,9 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|Content-Type: text/plain
|
|XYZ
- |--12345--""".stripMarginWithNewline("\r\n"))).to[MultipartByteRanges] should haveParts(
- BodyPart(HttpEntity(ContentTypes.`text/plain`, "ABC"), List(`Content-Range`(ContentRange(0, 2, 26)))),
- BodyPart(HttpEntity(ContentTypes.`text/plain`, "XYZ"), List(`Content-Range`(ContentRange(23, 25, 26)))))
+ |--12345--""".stripMarginWithNewline("\r\n"))).to[Multipart.ByteRanges] should haveParts(
+ Multipart.ByteRanges.BodyPart.Strict(ContentRange(0, 2, 26), HttpEntity(ContentTypes.`text/plain`, "ABC")),
+ Multipart.ByteRanges.BodyPart.Strict(ContentRange(23, 25, 26), HttpEntity(ContentTypes.`text/plain`, "XYZ")))
}
"multipartFormDataUnmarshaller should correctly unmarshal 'multipart/form-data' content" - {
@@ -124,29 +161,37 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|content-disposition: form-data; name=email
|
|test@there.com
- |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[MultipartFormData] should haveFormData(
- "email" -> BodyPart(HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com"), "email"))
+ |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[Multipart.FormData] should haveParts(
+ Multipart.FormData.BodyPart.Strict("email", HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com")))
}
"with a file" in {
- Unmarshal(HttpEntity(`multipart/form-data` withBoundary "XYZABC",
- """--XYZABC
- |Content-Disposition: form-data; name="email"
- |
- |test@there.com
- |--XYZABC
- |Content-Disposition: form-data; name="userfile"; filename="test.dat"
- |Content-Type: application/pdf
- |Content-Transfer-Encoding: binary
- |
- |filecontent
- |--XYZABC--""".stripMarginWithNewline("\r\n"))).to[StrictMultipartFormData] should haveFormData(
- "email" -> BodyPart(
- HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com"),
- List(`Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "email")))),
- "userfile" -> BodyPart(
- HttpEntity(MediaTypes.`application/pdf`, "filecontent"),
- List(RawHeader("Content-Transfer-Encoding", "binary"),
- `Content-Disposition`(ContentDispositionTypes.`form-data`, Map("name" -> "userfile", "filename" -> "test.dat")))))
+ Unmarshal {
+ HttpEntity.Default(
+ contentType = `multipart/form-data` withBoundary "XYZABC",
+ contentLength = 1, // not verified during unmarshalling
+ data = Source {
+ List(
+ ByteString {
+ """--XYZABC
+ |Content-Disposition: form-data; name="email"
+ |
+ |test@there.com
+ |--XYZABC
+ |Content-Dispo""".stripMarginWithNewline("\r\n")
+ },
+ ByteString {
+ """sition: form-data; name="userfile"; filename="test.dat"
+ |Content-Type: application/pdf
+ |Content-Transfer-Encoding: binary
+ |
+ |filecontent
+ |--XYZABC--""".stripMarginWithNewline("\r\n")
+ })
+ })
+ }.to[Multipart.FormData].flatMap(_.toStrict(1.second)) should haveParts(
+ Multipart.FormData.BodyPart.Strict("email", HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com")),
+ Multipart.FormData.BodyPart.Strict("userfile", HttpEntity(MediaTypes.`application/pdf`, "filecontent"),
+ Map("filename" -> "test.dat"), List(RawHeader("Content-Transfer-Encoding", "binary"))))
}
// TODO: reactivate after multipart/form-data unmarshalling integrity verification is implemented
//
@@ -171,24 +216,10 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
def evaluateTo[T](value: T): Matcher[Future[T]] =
equal(value).matcher[T] compose (x ⇒ Await.result(x, 1.second))
- def haveParts[T <: MultipartParts](parts: BodyPart*): Matcher[Future[T]] =
- equal(parts).matcher[Seq[BodyPart]] compose { x ⇒
+ def haveParts[T <: Multipart](parts: Multipart.BodyPart*): Matcher[Future[T]] =
+ equal(parts).matcher[Seq[Multipart.BodyPart]] compose { x ⇒
Await.result(x
.fast.flatMap(x ⇒ x.parts.grouped(100).runWith(Sink.future))
.fast.recover { case _: NoSuchElementException ⇒ Nil }, 1.second)
}
-
- def haveFormData(fields: (String, BodyPart)*): Matcher[Future[MultipartFormData]] =
- equal(fields).matcher[Seq[(String, BodyPart)]] compose { x ⇒
- Await.result(x
- .fast.flatMap(x ⇒ x.parts.grouped(100).runWith(Sink.future))
- .fast.recover { case _: NoSuchElementException ⇒ Nil }
- .fast.map {
- _ map { part ⇒
- part.headers.collectFirst {
- case `Content-Disposition`(ContentDispositionTypes.`form-data`, params) ⇒ params("name")
- }.get -> part
- }
- }, 1.second)
- }
}
diff --git a/akka-http/src/main/scala/akka/http/marshalling/Marshallers.scala b/akka-http/src/main/scala/akka/http/marshalling/Marshallers.scala
index 6650e2934c..36c7361178 100644
--- a/akka-http/src/main/scala/akka/http/marshalling/Marshallers.scala
+++ b/akka-http/src/main/scala/akka/http/marshalling/Marshallers.scala
@@ -27,10 +27,8 @@ object Marshallers extends SingleMarshallerMarshallers {
Marshallers(f(first) +: vector)
}
- implicit val NodeSeqMarshallers: ToEntityMarshallers[NodeSeq] = {
- import scala.concurrent.ExecutionContext.Implicits.global
+ implicit def nodeSeqMarshallers(implicit ec: ExecutionContext): ToEntityMarshallers[NodeSeq] =
Marshallers(`text/xml`, `application/xml`, `text/html`, `application/xhtml+xml`)(PredefinedToEntityMarshallers.nodeSeqMarshaller)
- }
implicit def entity2response[T](implicit m: Marshallers[T, ResponseEntity], ec: ExecutionContext): Marshallers[T, HttpResponse] =
m map (entity ⇒ HttpResponse(entity = entity))
diff --git a/akka-http/src/main/scala/akka/http/marshalling/MultipartMarshallers.scala b/akka-http/src/main/scala/akka/http/marshalling/MultipartMarshallers.scala
index 53bc3e1174..15dc65e10f 100644
--- a/akka-http/src/main/scala/akka/http/marshalling/MultipartMarshallers.scala
+++ b/akka-http/src/main/scala/akka/http/marshalling/MultipartMarshallers.scala
@@ -4,17 +4,15 @@
package akka.http.marshalling
-import scala.concurrent.ExecutionContext
+import akka.event.{ NoLogging, LoggingAdapter }
+
import scala.concurrent.forkjoin.ThreadLocalRandom
-import akka.actor.ActorRefFactory
import akka.parboiled2.util.Base64
import akka.stream.FlattenStrategy
import akka.stream.scaladsl._
import akka.http.engine.rendering.BodyPartRenderer
-import akka.http.util.actorSystem
-import akka.http.util.FastFuture._
+import akka.http.util.FastFuture
import akka.http.model._
-import MediaTypes._
trait MultipartMarshallers {
protected val multipartBoundaryRandom: java.util.Random = ThreadLocalRandom.current()
@@ -28,31 +26,23 @@ trait MultipartMarshallers {
Base64.custom.encodeToString(array, false)
}
- implicit def multipartByteRangesMarshaller(implicit refFactory: ActorRefFactory): ToEntityMarshaller[MultipartByteRanges] =
- multipartPartsMarshaller[MultipartByteRanges](`multipart/byteranges`)(refFactory)
- implicit def multipartContentMarshaller(implicit refFactory: ActorRefFactory): ToEntityMarshaller[MultipartContent] =
- multipartPartsMarshaller[MultipartContent](`multipart/mixed`)(refFactory)
-
- private def multipartPartsMarshaller[T <: MultipartParts](mediaType: MultipartMediaType)(implicit refFactory: ActorRefFactory): ToEntityMarshaller[T] = {
- val boundary = randomBoundary
- val mediaTypeWithBoundary = mediaType withBoundary boundary
- Marshaller.withOpenCharset(mediaTypeWithBoundary) { (value, charset) ⇒
- val log = actorSystem(refFactory).log
- val bodyPartRenderer = new BodyPartRenderer(boundary, charset.nioCharset, partHeadersSizeHint = 128, log)
- val chunks = value.parts.transform("bodyPartRenderer", () ⇒ bodyPartRenderer).flatten(FlattenStrategy.concat)
- HttpEntity.Chunked(ContentType(mediaTypeWithBoundary), chunks)
- }
- }
-
- implicit def multipartFormDataMarshaller(implicit mcm: ToEntityMarshaller[MultipartContent],
- ec: ExecutionContext): ToEntityMarshaller[MultipartFormData] =
+ implicit def multipartMarshaller[T <: Multipart](implicit log: LoggingAdapter = NoLogging): ToEntityMarshaller[T] =
Marshaller { value ⇒
- mcm(MultipartContent(value.parts)).fast.map {
- case Marshalling.WithOpenCharset(mt, marshal) ⇒
- val mediaType = `multipart/form-data` withBoundary mt.params("boundary")
- 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)
+ val boundary = randomBoundary
+ val contentType = ContentType(value.mediaType withBoundary boundary)
+ FastFuture.successful {
+ Marshalling.WithOpenCharset(contentType.mediaType, { charset ⇒
+ value match {
+ case x: Multipart.Strict ⇒
+ val data = BodyPartRenderer.strict(x.strictParts, boundary, charset.nioCharset, partHeadersSizeHint = 128, log)
+ HttpEntity(contentType, data)
+ case _ ⇒
+ val chunks = value.parts
+ .transform("bodyPartRenderer", () ⇒ BodyPartRenderer.streamed(boundary, charset.nioCharset, partHeadersSizeHint = 128, log))
+ .flatten(FlattenStrategy.concat)
+ HttpEntity.Chunked(contentType, chunks)
+ }
+ })
}
}
}
diff --git a/akka-http/src/main/scala/akka/http/unmarshalling/MultipartUnmarshallers.scala b/akka-http/src/main/scala/akka/http/unmarshalling/MultipartUnmarshallers.scala
index 51bb68b7d7..9b1a0b9b63 100644
--- a/akka-http/src/main/scala/akka/http/unmarshalling/MultipartUnmarshallers.scala
+++ b/akka-http/src/main/scala/akka/http/unmarshalling/MultipartUnmarshallers.scala
@@ -4,65 +4,95 @@
package akka.http.unmarshalling
-import akka.actor.ActorRefFactory
-import akka.stream.FlowMaterializer
-import akka.stream.scaladsl._
+import scala.collection.immutable
+import scala.collection.immutable.VectorBuilder
+import scala.concurrent.ExecutionContext
+import akka.event.{ NoLogging, LoggingAdapter }
import akka.http.engine.parsing.BodyPartParser
import akka.http.model._
import akka.http.util._
+import akka.stream.scaladsl._
import MediaRanges._
import MediaTypes._
import HttpCharsets._
trait MultipartUnmarshallers {
- implicit def defaultMultipartContentUnmarshaller(implicit refFactory: ActorRefFactory) = multipartContentUnmarshaller(`UTF-8`)
- def multipartContentUnmarshaller(defaultCharset: HttpCharset)(implicit refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartContent] =
- multipartPartsUnmarshaller[MultipartContent](`multipart/*`, ContentTypes.`text/plain` withCharset defaultCharset)(MultipartContent(_))
+ implicit def defaultMultipartGeneralUnmarshaller(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[Multipart.General] =
+ multipartGeneralUnmarshaller(`UTF-8`)
+ def multipartGeneralUnmarshaller(defaultCharset: HttpCharset)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[Multipart.General] =
+ multipartUnmarshaller[Multipart.General, Multipart.General.BodyPart, Multipart.General.BodyPart.Strict](
+ mediaRange = `multipart/*`,
+ defaultContentType = ContentTypes.`text/plain` withCharset defaultCharset,
+ createBodyPart = Multipart.General.BodyPart(_, _),
+ createStreamed = Multipart.General(_, _),
+ createStrictBodyPart = Multipart.General.BodyPart.Strict,
+ createStrict = Multipart.General.Strict)
- implicit def defaultMultipartByteRangesUnmarshaller(implicit refFactory: ActorRefFactory) = multipartByteRangesUnmarshaller(`UTF-8`)
- def multipartByteRangesUnmarshaller(defaultCharset: HttpCharset)(implicit refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartByteRanges] =
- multipartPartsUnmarshaller[MultipartByteRanges](`multipart/byteranges`,
- ContentTypes.`text/plain` withCharset defaultCharset)(MultipartByteRanges(_))
+ implicit def multipartFormDataUnmarshaller(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[Multipart.FormData] =
+ multipartUnmarshaller[Multipart.FormData, Multipart.FormData.BodyPart, Multipart.FormData.BodyPart.Strict](
+ mediaRange = `multipart/form-data`,
+ defaultContentType = ContentTypes.`application/octet-stream`,
+ createBodyPart = (entity, headers) ⇒ Multipart.General.BodyPart(entity, headers).toFormDataBodyPart.get,
+ createStreamed = (_, parts) ⇒ Multipart.FormData(parts),
+ createStrictBodyPart = (entity, headers) ⇒ Multipart.General.BodyPart.Strict(entity, headers).toFormDataBodyPart.get,
+ createStrict = (_, parts) ⇒ Multipart.FormData.Strict(parts))
- def multipartPartsUnmarshaller[T <: MultipartParts](mediaRange: MediaRange, defaultContentType: ContentType)(create: Source[BodyPart] ⇒ T)(implicit refFactory: ActorRefFactory): FromEntityUnmarshaller[T] =
+ implicit def defaultMultipartByteRangesUnmarshaller(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[Multipart.ByteRanges] =
+ multipartByteRangesUnmarshaller(`UTF-8`)
+ def multipartByteRangesUnmarshaller(defaultCharset: HttpCharset)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[Multipart.ByteRanges] =
+ multipartUnmarshaller[Multipart.ByteRanges, Multipart.ByteRanges.BodyPart, Multipart.ByteRanges.BodyPart.Strict](
+ mediaRange = `multipart/byteranges`,
+ defaultContentType = ContentTypes.`text/plain` withCharset defaultCharset,
+ createBodyPart = (entity, headers) ⇒ Multipart.General.BodyPart(entity, headers).toByteRangesBodyPart.get,
+ createStreamed = (_, parts) ⇒ Multipart.ByteRanges(parts),
+ createStrictBodyPart = (entity, headers) ⇒ Multipart.General.BodyPart.Strict(entity, headers).toByteRangesBodyPart.get,
+ createStrict = (_, parts) ⇒ Multipart.ByteRanges.Strict(parts))
+
+ def multipartUnmarshaller[T <: Multipart, BP <: Multipart.BodyPart, BPS <: Multipart.BodyPart.Strict](mediaRange: MediaRange,
+ defaultContentType: ContentType,
+ createBodyPart: (BodyPartEntity, List[HttpHeader]) ⇒ BP,
+ createStreamed: (MultipartMediaType, Source[BP]) ⇒ T,
+ createStrictBodyPart: (HttpEntity.Strict, List[HttpHeader]) ⇒ BPS,
+ createStrict: (MultipartMediaType, immutable.Seq[BPS]) ⇒ T)(implicit ec: ExecutionContext, log: LoggingAdapter = NoLogging): FromEntityUnmarshaller[T] =
Unmarshaller { entity ⇒
- if (mediaRange matches entity.contentType.mediaType) {
+ if (entity.contentType.mediaType.isMultipart && mediaRange.matches(entity.contentType.mediaType)) {
entity.contentType.mediaType.params.get("boundary") match {
- case None ⇒ FastFuture.failed(UnmarshallingError.InvalidContent("Content-Type with a multipart media type must have a 'boundary' parameter"))
+ case None ⇒
+ FastFuture.failed(UnmarshallingError.InvalidContent("Content-Type with a multipart media type must have a 'boundary' parameter"))
case Some(boundary) ⇒
- val bodyParts = entity.dataBytes
- .transform("bodyPart", () ⇒ new BodyPartParser(defaultContentType, boundary, actorSystem(refFactory).log))
- .splitWhen(_.isInstanceOf[BodyPartParser.BodyPartStart])
- .headAndTail
- .collect {
- case (BodyPartParser.BodyPartStart(headers, createEntity), entityParts) ⇒
- BodyPart(createEntity(entityParts), headers)
- case (BodyPartParser.ParseError(errorInfo), _) ⇒ throw new ParsingException(errorInfo)
+ import BodyPartParser._
+ val parser = new BodyPartParser(defaultContentType, boundary, log)
+ FastFuture.successful {
+ entity match {
+ case HttpEntity.Strict(ContentType(mediaType: MultipartMediaType, _), data) ⇒
+ val builder = new VectorBuilder[BPS]()
+ (parser.onNext(data) ++ parser.onTermination(None)) foreach {
+ case BodyPartStart(headers, createEntity) ⇒
+ val entity = createEntity(Source.empty()) match {
+ case x: HttpEntity.Strict ⇒ x
+ case x ⇒ throw new IllegalStateException("Unexpected entity type from strict BodyPartParser: " + x.getClass.getName)
+ }
+ builder += createStrictBodyPart(entity, headers)
+ case ParseError(errorInfo) ⇒ throw new ParsingException(errorInfo)
+ case x ⇒ throw new IllegalStateException(s"Unexpected BodyPartParser result `x` in strict case")
+ }
+ createStrict(mediaType, builder.result())
+ case _ ⇒
+ val bodyParts = entity.dataBytes
+ .transform("bodyPart", () ⇒ parser)
+ .splitWhen(_.isInstanceOf[BodyPartStart])
+ .headAndTail
+ .collect {
+ case (BodyPartStart(headers, createEntity), entityParts) ⇒ createBodyPart(createEntity(entityParts), headers)
+ case (ParseError(errorInfo), _) ⇒ throw new ParsingException(errorInfo)
+ }
+ createStreamed(entity.contentType.mediaType.asInstanceOf[MultipartMediaType], bodyParts)
}
- FastFuture.successful(create(bodyParts))
+ }
}
} else FastFuture.failed(UnmarshallingError.UnsupportedContentType(ContentTypeRange(mediaRange) :: Nil))
}
-
- implicit def defaultMultipartFormDataUnmarshaller(implicit refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
- multipartFormDataUnmarshaller(verifyIntegrity = true)
- def multipartFormDataUnmarshaller(verifyIntegrity: Boolean = true)(implicit refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
- multipartPartsUnmarshaller(`multipart/form-data`, ContentTypes.`application/octet-stream`) { bodyParts ⇒
- def verify(part: BodyPart): BodyPart = part // TODO
- val parts = if (verifyIntegrity) bodyParts.map(verify) else bodyParts
- MultipartFormData(parts)
- }
-
- implicit def defaultStrictMultipartFormDataUnmarshaller(implicit fm: FlowMaterializer,
- refFactory: ActorRefFactory): FromEntityUnmarshaller[StrictMultipartFormData] =
- strictMultipartFormDataUnmarshaller(verifyIntegrity = true)
- def strictMultipartFormDataUnmarshaller(verifyIntegrity: Boolean = true)(implicit fm: FlowMaterializer,
- refFactory: ActorRefFactory): FromEntityUnmarshaller[StrictMultipartFormData] = {
- implicit val ec = actorSystem(refFactory).dispatcher
- multipartFormDataUnmarshaller(verifyIntegrity) flatMap (mfd ⇒ mfd.toStrict())
- }
-
}
object MultipartUnmarshallers extends MultipartUnmarshallers
\ No newline at end of file