=htp Apply remaining review feedback
This commit is contained in:
parent
3747196356
commit
e7ef23f6f8
6 changed files with 89 additions and 40 deletions
|
|
@ -5,11 +5,15 @@
|
||||||
package akka.http.model
|
package akka.http.model
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
|
import akka.stream.FlowMaterializer
|
||||||
|
import akka.stream.scaladsl.Flow
|
||||||
import org.reactivestreams.Publisher
|
import org.reactivestreams.Publisher
|
||||||
import akka.stream.impl.SynchronousPublisherFromIterable
|
import akka.stream.impl.SynchronousPublisherFromIterable
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import headers._
|
import headers._
|
||||||
|
|
||||||
|
import scala.concurrent.{ ExecutionContext, Future }
|
||||||
|
|
||||||
trait MultipartParts {
|
trait MultipartParts {
|
||||||
def parts: Publisher[BodyPart]
|
def parts: Publisher[BodyPart]
|
||||||
}
|
}
|
||||||
|
|
@ -47,8 +51,26 @@ object MultipartByteRanges {
|
||||||
* All parts must contain a Content-Disposition header with a type form-data
|
* All parts must contain a Content-Disposition header with a type form-data
|
||||||
* and a name parameter that is unique.
|
* and a name parameter that is unique.
|
||||||
*/
|
*/
|
||||||
final case class MultipartFormData(parts: Publisher[BodyPart]) extends MultipartParts {
|
case class MultipartFormData(parts: Publisher[BodyPart]) extends MultipartParts {
|
||||||
// def get(partName: String): Option[BodyPart] = fields.find(_.name.exists(_ == partName))
|
/**
|
||||||
|
* Turns this instance into its strict specialization using the given `maxFieldCount` as the field number cut-off
|
||||||
|
* hint.
|
||||||
|
*/
|
||||||
|
def toStrict(materializer: FlowMaterializer, maxFieldCount: Int = 1000)(implicit ec: ExecutionContext): Future[StrictMultipartFormData] =
|
||||||
|
Flow(parts).grouped(maxFieldCount).toFuture(materializer).map(new StrictMultipartFormData(_))
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A specialized `MultipartFormData` that allows full random access to its parts.
|
||||||
|
*/
|
||||||
|
class StrictMultipartFormData(val fields: immutable.Seq[BodyPart]) extends MultipartFormData(SynchronousPublisherFromIterable(fields)) {
|
||||||
|
/**
|
||||||
|
* Returns the BodyPart with the given name, if found.
|
||||||
|
*/
|
||||||
|
def get(partName: String): Option[BodyPart] = fields.find(_.name.exists(_ == partName))
|
||||||
|
|
||||||
|
override def toStrict(materializer: FlowMaterializer, maxFieldCount: Int)(implicit ec: ExecutionContext): Future[StrictMultipartFormData] =
|
||||||
|
Future.successful(this)
|
||||||
}
|
}
|
||||||
|
|
||||||
object MultipartFormData {
|
object MultipartFormData {
|
||||||
|
|
|
||||||
|
|
@ -37,12 +37,15 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
||||||
|
|
||||||
sealed trait StateResult // phantom type for ensuring soundness of our parsing method setup
|
sealed trait StateResult // phantom type for ensuring soundness of our parsing method setup
|
||||||
|
|
||||||
private[this] val needle = new Array[Byte](boundary.length + 4)
|
private[this] val needle: Array[Byte] = {
|
||||||
needle(0) = '\r'.toByte
|
val array = new Array[Byte](boundary.length + 4)
|
||||||
needle(1) = '\n'.toByte
|
array(0) = '\r'.toByte
|
||||||
needle(2) = '-'.toByte
|
array(1) = '\n'.toByte
|
||||||
needle(3) = '-'.toByte
|
array(2) = '-'.toByte
|
||||||
boundary.getAsciiBytes(needle, 4)
|
array(3) = '-'.toByte
|
||||||
|
boundary.getAsciiBytes(array, 4)
|
||||||
|
array
|
||||||
|
}
|
||||||
|
|
||||||
// we use the Boyer-Moore string search algorithm for finding the boundaries in the multipart entity,
|
// we use the Boyer-Moore string search algorithm for finding the boundaries in the multipart entity,
|
||||||
// TODO: evaluate whether an upgrade to the more efficient FJS is worth the implementation cost
|
// TODO: evaluate whether an upgrade to the more efficient FJS is worth the implementation cost
|
||||||
|
|
@ -64,13 +67,14 @@ private[http] final class BodyPartParser(defaultContentType: ContentType,
|
||||||
try state(input)
|
try state(input)
|
||||||
catch {
|
catch {
|
||||||
case e: ParsingException ⇒ fail(e.info)
|
case e: ParsingException ⇒ fail(e.info)
|
||||||
case NotEnoughDataException ⇒ throw new IllegalStateException // we are missing a try/catch{continue} wrapper somewhere
|
case NotEnoughDataException ⇒ throw new IllegalStateException(NotEnoughDataException) // we are missing a try/catch{continue} wrapper somewhere
|
||||||
}
|
}
|
||||||
result.toList
|
result.toList
|
||||||
}
|
}
|
||||||
|
|
||||||
def tryParseInitialBoundary(input: ByteString): StateResult = {
|
def tryParseInitialBoundary(input: ByteString): StateResult = {
|
||||||
// we don't use boyerMoore here because we are looking for the boundary *without* a preceding CRLF
|
// we don't use boyerMoore here because we are testing for the boundary *without* a
|
||||||
|
// preceding CRLF and at a known location (the very beginning of the entity)
|
||||||
try {
|
try {
|
||||||
@tailrec def rec(ix: Int): StateResult =
|
@tailrec def rec(ix: Int): StateResult =
|
||||||
if (ix < needle.length) {
|
if (ix < needle.length) {
|
||||||
|
|
|
||||||
|
|
@ -4,7 +4,8 @@
|
||||||
|
|
||||||
package akka.http.marshalling
|
package akka.http.marshalling
|
||||||
|
|
||||||
import java.util.Random
|
import scala.concurrent.ExecutionContext
|
||||||
|
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||||
import akka.actor.ActorRefFactory
|
import akka.actor.ActorRefFactory
|
||||||
import akka.parboiled2.util.Base64
|
import akka.parboiled2.util.Base64
|
||||||
import akka.stream.{ FlattenStrategy, FlowMaterializer }
|
import akka.stream.{ FlattenStrategy, FlowMaterializer }
|
||||||
|
|
@ -14,10 +15,8 @@ import akka.http.util.actorSystem
|
||||||
import akka.http.model._
|
import akka.http.model._
|
||||||
import MediaTypes._
|
import MediaTypes._
|
||||||
|
|
||||||
import scala.concurrent.ExecutionContext
|
|
||||||
|
|
||||||
trait MultipartMarshallers {
|
trait MultipartMarshallers {
|
||||||
protected val multipartBoundaryRandom = new Random
|
protected val multipartBoundaryRandom: java.util.Random = ThreadLocalRandom.current()
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new random 144-bit number and base64 encodes it (using a custom "safe" alphabet, yielding 24 characters).
|
* Creates a new random 144-bit number and base64 encodes it (using a custom "safe" alphabet, yielding 24 characters).
|
||||||
|
|
|
||||||
|
|
@ -56,14 +56,30 @@ trait MultipartUnmarshallers {
|
||||||
|
|
||||||
implicit def defaultMultipartFormDataUnmarshaller(implicit fm: FlowMaterializer,
|
implicit def defaultMultipartFormDataUnmarshaller(implicit fm: FlowMaterializer,
|
||||||
refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
|
refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
|
||||||
multipartFormDataUnmarshaller(strict = true)
|
multipartFormDataUnmarshaller(verifyIntegrity = true)
|
||||||
def multipartFormDataUnmarshaller(strict: Boolean = true)(implicit fm: FlowMaterializer,
|
def multipartFormDataUnmarshaller(verifyIntegrity: Boolean = true)(implicit fm: FlowMaterializer,
|
||||||
refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
|
refFactory: ActorRefFactory): FromEntityUnmarshaller[MultipartFormData] =
|
||||||
multipartPartsUnmarshaller(`multipart/form-data`, ContentTypes.`application/octet-stream`) { bodyParts ⇒
|
multipartPartsUnmarshaller(`multipart/form-data`, ContentTypes.`application/octet-stream`) { bodyParts ⇒
|
||||||
def verify(part: BodyPart): BodyPart = part // TODO
|
def verify(part: BodyPart): BodyPart = part // TODO
|
||||||
val parts = if (strict) Flow(bodyParts).map(verify).toPublisher(fm) else bodyParts
|
val parts = if (verifyIntegrity) Flow(bodyParts).map(verify).toPublisher(fm) else bodyParts
|
||||||
MultipartFormData(parts)
|
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
|
||||||
|
val m = multipartFormDataUnmarshaller(verifyIntegrity)
|
||||||
|
Unmarshaller {
|
||||||
|
m(_) flatMap {
|
||||||
|
case Unmarshalling.Success(mfd) ⇒ mfd.toStrict(fm).map(Unmarshalling.Success.apply)
|
||||||
|
case e: Unmarshalling.Failure ⇒ Future.successful(e)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object MultipartUnmarshallers extends MultipartUnmarshallers
|
object MultipartUnmarshallers extends MultipartUnmarshallers
|
||||||
|
|
@ -42,6 +42,7 @@ object Unmarshaller
|
||||||
sealed trait Unmarshalling[+A] {
|
sealed trait Unmarshalling[+A] {
|
||||||
def isSuccess: Boolean
|
def isSuccess: Boolean
|
||||||
def isFailure: Boolean
|
def isFailure: Boolean
|
||||||
|
def value: A
|
||||||
def map[B](f: A ⇒ B): Unmarshalling[B]
|
def map[B](f: A ⇒ B): Unmarshalling[B]
|
||||||
def flatMap[B](f: A ⇒ Unmarshalling[B]): Unmarshalling[B]
|
def flatMap[B](f: A ⇒ Unmarshalling[B]): Unmarshalling[B]
|
||||||
def recover[AA >: A](f: PartialFunction[Unmarshalling.Failure, AA]): Unmarshalling[AA]
|
def recover[AA >: A](f: PartialFunction[Unmarshalling.Failure, AA]): Unmarshalling[AA]
|
||||||
|
|
@ -59,6 +60,7 @@ object Unmarshalling {
|
||||||
sealed abstract class Failure extends Unmarshalling[Nothing] {
|
sealed abstract class Failure extends Unmarshalling[Nothing] {
|
||||||
def isSuccess = false
|
def isSuccess = false
|
||||||
def isFailure = true
|
def isFailure = true
|
||||||
|
def value = sys.error("Expected Unmarshalling.Success but got " + this)
|
||||||
def map[B](f: Nothing ⇒ B) = this
|
def map[B](f: Nothing ⇒ B) = this
|
||||||
def flatMap[B](f: Nothing ⇒ Unmarshalling[B]) = this
|
def flatMap[B](f: Nothing ⇒ Unmarshalling[B]) = this
|
||||||
def recover[AA >: Nothing](f: PartialFunction[Unmarshalling.Failure, AA]) =
|
def recover[AA >: Nothing](f: PartialFunction[Unmarshalling.Failure, AA]) =
|
||||||
|
|
|
||||||
|
|
@ -4,6 +4,7 @@
|
||||||
|
|
||||||
package akka.http
|
package akka.http
|
||||||
|
|
||||||
|
import scala.xml.NodeSeq
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.{ Future, Await }
|
import scala.concurrent.{ Future, Await }
|
||||||
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
|
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
|
||||||
|
|
@ -30,9 +31,9 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
||||||
"charArrayUnmarshaller should unmarshal `text/plain` content in UTF-8 to char arrays" in {
|
"charArrayUnmarshaller should unmarshal `text/plain` content in UTF-8 to char arrays" in {
|
||||||
Unmarshal(HttpEntity("árvíztűrő ütvefúrógép")).to[Array[Char]] should evaluateTo("árvíztűrő ütvefúrógép".toCharArray)
|
Unmarshal(HttpEntity("árvíztűrő ütvefúrógép")).to[Array[Char]] should evaluateTo("árvíztűrő ütvefúrógép".toCharArray)
|
||||||
}
|
}
|
||||||
// "nodeSeqUnmarshaller should unmarshal `text/xml` content in UTF-8 to NodeSeqs" in {
|
"nodeSeqUnmarshaller should unmarshal `text/xml` content in UTF-8 to NodeSeqs" in {
|
||||||
// Unmarshal(HttpEntity(`text/xml`, "<int>Hällö</int>")).to[NodeSeq].map(_.map(_.text)) shouldEqual "Hällö"
|
Unmarshal(HttpEntity(`text/xml`, "<int>Hällö</int>")).to[NodeSeq].map(_.map(_.text)) should evaluateTo("Hällö")
|
||||||
// }
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"The MultipartUnmarshallers." - {
|
"The MultipartUnmarshallers." - {
|
||||||
|
|
@ -116,26 +117,31 @@ class UnmarshallingSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
|
||||||
|
|
|
|
||||||
|test@there.com
|
|test@there.com
|
||||||
|--XYZABC--""".stripMarginWithNewline("\r\n"))).to[MultipartFormData] should haveFormData(
|
|--XYZABC--""".stripMarginWithNewline("\r\n"))).to[MultipartFormData] should haveFormData(
|
||||||
"email" -> BodyPart(
|
"email" -> BodyPart(HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com"), "email"))
|
||||||
HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com"), "email"))
|
|
||||||
}
|
}
|
||||||
// "with a file" in {
|
"with a file" in {
|
||||||
// HttpEntity(`multipart/form-data` withBoundary "XYZABC",
|
Unmarshal(HttpEntity(`multipart/form-data` withBoundary "XYZABC",
|
||||||
// """|--XYZABC
|
"""--XYZABC
|
||||||
// |Content-Disposition: form-data; name="email"
|
|Content-Disposition: form-data; name="email"
|
||||||
// |
|
|
|
||||||
// |test@there.com
|
|test@there.com
|
||||||
// |--XYZABC
|
|--XYZABC
|
||||||
// |Content-Disposition: form-data; name="userfile"; filename="test.dat"
|
|Content-Disposition: form-data; name="userfile"; filename="test.dat"
|
||||||
// |Content-Type: application/octet-stream
|
|Content-Type: application/pdf
|
||||||
// |Content-Transfer-Encoding: binary
|
|Content-Transfer-Encoding: binary
|
||||||
// |
|
|
|
||||||
// |filecontent
|
|filecontent
|
||||||
// |--XYZABC--""".stripMargin).as[MultipartFormData].get.fields.map {
|
|--XYZABC--""".stripMarginWithNewline("\r\n"))).to[StrictMultipartFormData] should haveFormData(
|
||||||
// case part @ BodyPart(entity, _) ⇒
|
"email" -> BodyPart(
|
||||||
// part.name.get + ": " + entity.as[String].get + part.filename.map(",filename: " + _).getOrElse("")
|
HttpEntity(ContentTypes.`application/octet-stream`, "test@there.com"),
|
||||||
// }.mkString("|") === "email: test@there.com|userfile: filecontent,filename: test.dat"
|
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")))))
|
||||||
|
}
|
||||||
|
// TODO: reactivate after multipart/form-data unmarshalling integrity verification is implemented
|
||||||
|
//
|
||||||
// "reject illegal multipart content" in {
|
// "reject illegal multipart content" in {
|
||||||
// val Left(MalformedContent(msg, _)) = HttpEntity(`multipart/form-data` withBoundary "XYZABC", "--noboundary--").as[MultipartFormData]
|
// val Left(MalformedContent(msg, _)) = HttpEntity(`multipart/form-data` withBoundary "XYZABC", "--noboundary--").as[MultipartFormData]
|
||||||
// msg === "Missing start boundary"
|
// msg === "Missing start boundary"
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue