Skip to content

Commit

Permalink
only recover from mising data errors
Browse files Browse the repository at this point in the history
  • Loading branch information
adamgfraser committed Oct 10, 2023
1 parent de31152 commit 86db818
Show file tree
Hide file tree
Showing 7 changed files with 135 additions and 46 deletions.
83 changes: 66 additions & 17 deletions zio-http/src/main/scala/zio/http/codec/HttpCodec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ sealed trait HttpCodec[-AtomTypes, Value] {
if (self eq HttpCodec.Halt) that.asInstanceOf[HttpCodec[AtomTypes1, alternator.Out]]
else {
HttpCodec
.Fallback(self, that)
.Fallback(self, that, HttpCodec.Fallback.Condition.IsHttpCodecError)
.transform[alternator.Out](either => either.fold(alternator.left(_), alternator.right(_)))(value =>
alternator
.unleft(value)
Expand Down Expand Up @@ -144,7 +144,8 @@ sealed trait HttpCodec[-AtomTypes, Value] {
* inside the returned collection is guaranteed to contain no nested
* alternatives.
*/
final def alternatives: Chunk[HttpCodec[AtomTypes, Value]] = HttpCodec.flattenFallbacks(self)
final def alternatives: Chunk[(HttpCodec[AtomTypes, Value], HttpCodec.Fallback.Condition)] =
HttpCodec.flattenFallbacks(self)

/**
* Returns a new codec that is the same as this one, but has attached
Expand Down Expand Up @@ -271,9 +272,12 @@ sealed trait HttpCodec[-AtomTypes, Value] {
*/
final def optional: HttpCodec[AtomTypes, Option[Value]] =
Annotated(
self
.orElseEither(HttpCodec.empty)
.transform(_.swap.toOption)(_.fold[Either[Unit, Value]](Left(()))(Right(_)).swap),
if (self eq HttpCodec.Halt) HttpCodec.empty.asInstanceOf[HttpCodec[AtomTypes, Option[Value]]]
else {
HttpCodec
.Fallback(self, HttpCodec.empty, HttpCodec.Fallback.Condition.isMissingDataOnly)
.transform[Option[Value]](either => either.fold(Some(_), _ => None))(_.toLeft(()))
},
Metadata.Optional(),
)

Expand Down Expand Up @@ -654,37 +658,82 @@ object HttpCodec extends ContentCodecs with HeaderCodecs with MethodCodecs with
private[http] final case class Fallback[AtomType, A, B](
left: HttpCodec[AtomType, A],
right: HttpCodec[AtomType, B],
condition: Fallback.Condition,
) extends HttpCodec[AtomType, Either[A, B]] {
type Left = A
type Right = B
type Out = Either[A, B]
}

private[http] object Fallback {

/**
* `Condition` describes the circumstances under which the `right` codec in
* a `Fallback` is willing to attempt to recover from a failure of the
* `left` codec. All implementations of `Fallback` other than `optional` are
* willing to attempt to recover from any `HttpCodecError`. Implementations
* of `Fallback` constructed from `optional` are only willing to attempt to
* recover from `MissingHeader` or `MissingQueryParam` errors.
*/
sealed trait Condition { self =>
def apply(cause: Cause[Any]): Boolean =
self match {
case Condition.IsHttpCodecError => HttpCodecError.isHttpCodecError(cause)
case Condition.isMissingDataOnly => HttpCodecError.isMissingDataOnly(cause)
}
def combine(that: Condition): Condition =
(self, that) match {
case (Condition.isMissingDataOnly, _) => Condition.isMissingDataOnly
case (_, Condition.isMissingDataOnly) => Condition.isMissingDataOnly
case _ => Condition.IsHttpCodecError
}
def isHttpCodecError: Boolean = self match {
case Condition.IsHttpCodecError => true
case _ => false
}
def isMissingDataOnly: Boolean = self match {
case Condition.isMissingDataOnly => true
case _ => false
}
}
object Condition {
case object IsHttpCodecError extends Condition
case object isMissingDataOnly extends Condition
}
}

private[http] def flattenFallbacks[AtomTypes, A](
api: HttpCodec[AtomTypes, A],
): Chunk[HttpCodec[AtomTypes, A]] = {
def rewrite[T, B](api: HttpCodec[T, B]): Chunk[HttpCodec[T, B]] =
): Chunk[(HttpCodec[AtomTypes, A], Fallback.Condition)] = {
def rewrite[T, B](api: HttpCodec[T, B]): Chunk[(HttpCodec[T, B], Fallback.Condition)] =
api match {
case fallback @ HttpCodec.Fallback(left, right) =>
rewrite[T, fallback.Left](left).map(_.toLeft[fallback.Right]) ++ rewrite[T, fallback.Right](right)
.map(_.toRight[fallback.Left])
case fallback @ HttpCodec.Fallback(left, right, condition) =>
rewrite[T, fallback.Left](left).map { case (codec, condition) =>
codec.toLeft[fallback.Right] -> condition
} ++
rewrite[T, fallback.Right](right).map { case (codec, _) =>
codec.toRight[fallback.Left] -> condition
}

case transform @ HttpCodec.TransformOrFail(codec, f, g) =>
rewrite[T, transform.In](codec).map(HttpCodec.TransformOrFail(_, f, g))
rewrite[T, transform.In](codec).map { case (codec, condition) =>
HttpCodec.TransformOrFail(codec, f, g) -> condition
}

case combine @ HttpCodec.Combine(left, right, combiner) =>
for {
l <- rewrite[T, combine.Left](left)
r <- rewrite[T, combine.Right](right)
} yield HttpCodec.Combine(l, r, combiner)
(l, lCondition) <- rewrite[T, combine.Left](left)
(r, rCondition) <- rewrite[T, combine.Right](right)
} yield HttpCodec.Combine(l, r, combiner) -> lCondition.combine(rCondition)

case HttpCodec.Annotated(in, metadata) => rewrite[T, B](in).map(_.annotate(metadata))
case HttpCodec.Annotated(in, metadata) =>
rewrite[T, B](in).map { case (codec, missingDataOnly) => codec.annotate(metadata) -> missingDataOnly }

case HttpCodec.Empty => Chunk.single(HttpCodec.Empty)
case HttpCodec.Empty => Chunk.single(HttpCodec.Empty -> Fallback.Condition.IsHttpCodecError)

case HttpCodec.Halt => Chunk.empty

case atom: Atom[_, _] => Chunk.single(atom)
case atom: Atom[_, _] => Chunk.single(atom -> Fallback.Condition.IsHttpCodecError)
}

rewrite(api)
Expand Down
5 changes: 5 additions & 0 deletions zio-http/src/main/scala/zio/http/codec/HttpCodecError.scala
Original file line number Diff line number Diff line change
Expand Up @@ -76,4 +76,9 @@ object HttpCodecError {
def isHttpCodecError(cause: Cause[Any]): Boolean = {
!cause.isFailure && cause.defects.forall(e => e.isInstanceOf[HttpCodecError])
}

def isMissingDataOnly(cause: Cause[Any]): Boolean =
!cause.isFailure && cause.defects.forall(e =>
e.isInstanceOf[HttpCodecError.MissingHeader] || e.isInstanceOf[HttpCodecError.MissingQueryParam],
)
}
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,6 @@ private[http] object AtomizedCodecs {
case Annotated(api, _) => flattenedAtoms(api)
case Empty => Chunk.empty
case Halt => Chunk.empty
case Fallback(_, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
case Fallback(_, _, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,30 +44,34 @@ private[codec] object EncoderDecoder {

flattened.length match {
case 0 => Undefined()
case 1 => Single(flattened.head, mediaType)
case 1 => Single(flattened.head._1, mediaType)
case _ => Multiple(flattened)
}
}

private final case class Multiple[-AtomTypes, Value](httpCodecs: Chunk[HttpCodec[AtomTypes, Value]])
extends EncoderDecoder[AtomTypes, Value] {
val singles = httpCodecs.map(Single(_))
private final case class Multiple[-AtomTypes, Value](
httpCodecs: Chunk[(HttpCodec[AtomTypes, Value], HttpCodec.Fallback.Condition)],
) extends EncoderDecoder[AtomTypes, Value] {
val singles = httpCodecs.map { case (httpCodec, condition) => Single(httpCodec) -> condition }

def decode(url: URL, status: Status, method: Method, headers: Headers, body: Body)(implicit
trace: Trace,
): Task[Value] = {
def tryDecode(i: Int, lastError: Cause[Throwable]): Task[Value] = {
if (i >= singles.length) ZIO.refailCause(lastError)
else {
val codec = singles(i)

codec
.decode(url, status, method, headers, body)
.catchAllCause(cause =>
if (HttpCodecError.isHttpCodecError(cause)) {
tryDecode(i + 1, lastError && cause)
} else ZIO.refailCause(cause),
)
val (codec, condition) = singles(i)

if (condition.isMissingDataOnly && !HttpCodecError.isMissingDataOnly(lastError))
tryDecode(i + 1, lastError)
else
codec
.decode(url, status, method, headers, body)
.catchAllCause(cause =>
if (HttpCodecError.isHttpCodecError(cause)) {
tryDecode(i + 1, lastError && cause)
} else ZIO.refailCause(cause),
)
}
}

Expand All @@ -80,7 +84,7 @@ private[codec] object EncoderDecoder {
var lastError = null.asInstanceOf[Throwable]

while (i < singles.length) {
val current = singles(i)
val (current, _) = singles(i)

try {
encoded = current.encodeWith(value)(f)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,7 @@ private[http] object Mechanic {
case Annotated(api, _) => indexedImpl(api.asInstanceOf[HttpCodec[R, A]], indices)
case Empty => (Empty.asInstanceOf[HttpCodec[R, A]], indices)
case Halt => (Halt.asInstanceOf[HttpCodec[R, A]], indices)
case Fallback(_, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
case Fallback(_, _, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
}

def makeConstructor[R, A](
Expand Down Expand Up @@ -101,7 +101,7 @@ private[http] object Mechanic {

case Halt => throw HaltException

case Fallback(_, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
case Fallback(_, _, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
}
}

Expand Down Expand Up @@ -143,7 +143,7 @@ private[http] object Mechanic {

case Halt => (_, _) => ()

case Fallback(_, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
case Fallback(_, _, _) => throw new UnsupportedOperationException("Cannot handle fallback at this level")
}
}
}
29 changes: 18 additions & 11 deletions zio-http/src/main/scala/zio/http/endpoint/Endpoint.scala
Original file line number Diff line number Diff line change
Expand Up @@ -66,9 +66,9 @@ final case class Endpoint[PathInput, Input, Err, Output, Middleware <: EndpointM
* Flattens out this endpoint to a chunk of alternatives. Each alternative is
* guaranteed to not have any alternatives itself.
*/
def alternatives: Chunk[Endpoint[PathInput, Input, Err, Output, Middleware]] =
self.input.alternatives.map { input =>
self.copy(input = input)
def alternatives: Chunk[(Endpoint[PathInput, Input, Err, Output, Middleware], HttpCodec.Fallback.Condition)] =
self.input.alternatives.map { case (input, condition) =>
self.copy(input = input) -> condition
}

def apply(input: Input): Invocation[PathInput, Input, Err, Output, Middleware] =
Expand Down Expand Up @@ -150,7 +150,7 @@ final case class Endpoint[PathInput, Input, Err, Output, Middleware <: EndpointM
def implement[Env](original: Handler[Env, Err, Input, Output])(implicit trace: Trace): Route[Env, Nothing] = {
import HttpCodecError.isHttpCodecError

val handlers = self.alternatives.map { endpoint =>
val handlers = self.alternatives.map { case (endpoint, condition) =>
Handler.fromFunctionZIO { (request: zio.http.Request) =>
val outputMediaTypes = request.headers
.get(Header.Accept)
Expand All @@ -161,21 +161,28 @@ final case class Endpoint[PathInput, Input, Err, Output, Middleware <: EndpointM
ZIO.succeed(endpoint.error.encodeResponse(error, outputMediaTypes))
}
}
}
} -> condition
}

// TODO: What to do if there are no endpoints??
val handlers2 =
NonEmptyChunk
.fromChunk(handlers)
.getOrElse(NonEmptyChunk(Handler.fail(zio.http.Response(status = Status.NotFound))))
.getOrElse(
NonEmptyChunk(
Handler.fail(zio.http.Response(status = Status.NotFound)) -> HttpCodec.Fallback.Condition.IsHttpCodecError,
),
)

val handler =
Handler.firstSuccessOf(handlers2, isHttpCodecError(_)).catchAllCause {
case cause if isHttpCodecError(cause) =>
Handler.succeed(zio.http.Response(status = Status.BadRequest))

case cause => Handler.failCause(cause)
handlers.tail.foldLeft(handlers2.head._1) { case (acc, (handler, condition)) =>
acc.catchAllCause { cause =>
if (condition(cause)) {
handler
} else {
Handler.failCause(cause)
}
}
}

Route.handled(self.route)(handler)
Expand Down
24 changes: 24 additions & 0 deletions zio-http/src/test/scala/zio/http/codec/HttpCodecSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,30 @@ object HttpCodecSpec extends ZIOHttpSpec {
} yield assertTrue(result.causeOption.get.defects.forall(_ == e))
}
} +
suite("optional") {
test("fallback for missing values") {
val codec = QueryCodec.query("name").transformOrFail[String](_ => Left("fail"))(Right(_))

val request = Request.get(url = URL.root)

val optional = codec.optional

for {
result <- optional.decodeRequest(request)
} yield assertTrue(result.isEmpty)
} +
test("no fallback for decoding errors") {
val codec = QueryCodec.query("key").transformOrFail[String](_ => Left("fail"))(Right(_))

val request = Request.get(url = URL.root.copy(queryParams = QueryParams("key" -> "value")))

val optional = codec.optional

for {
result <- optional.decodeRequest(request).exit
} yield assertTrue(result.isFailure)
}
} +
suite("HeaderCodec") {
test("dummy test") {
assertTrue(true)
Expand Down

0 comments on commit 86db818

Please sign in to comment.