Skip to content

Commit

Permalink
Error hierarchy (#411)
Browse files Browse the repository at this point in the history
* defined a new Decode Error hierarchy and propagated through current codecs

* removed all generic errors.

* formatted imports

* fixed type for MalformedFieldWithPath error

* fixed macro to properly identify the @valdiate annotation

* formatted

* formatted

* formatted

* formatted

* formatted
  • Loading branch information
devsprint authored Nov 7, 2022
1 parent 4cebfd1 commit 0e09815
Show file tree
Hide file tree
Showing 14 changed files with 225 additions and 159 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ object DeriveSchema {
tpe.typeSymbol.asClass.primaryConstructor.asMethod.paramLists.headOption.map { symbols =>
symbols.map { symbol =>
symbol.annotations.collect {
case annotation if (annotation.tree.tpe.toString.startsWith("_root_.zio.schema.annotation.validate")) =>
case annotation if (annotation.tree.tpe.toString.startsWith("zio.schema.annotation.validate")) =>
annotation.tree match {
case q"new $annConstructor(..$annotationArgs)" =>
q"..$annotationArgs"
Expand Down Expand Up @@ -616,10 +616,10 @@ object DeriveSchema {
case _ => c.abort(c.enclosingPosition, (s"Invalid type $tpe for @deriveSchema"))
}

val keySchema = q"""zio.schema.Schema[$keyType]"""
val valueSchema = q"""zio.schema.Schema[$valueType]"""
val keySchema = q"""_root_.zio.schema.Schema[$keyType]"""
val valueSchema = q"""_root_.zio.schema.Schema[$valueType]"""

q"""{lazy val $selfRefIdent: zio.schema.Schema.Map[$keyType, $valueType] = zio.schema.Schema.Map.apply[$keyType, $valueType]($keySchema, $valueSchema, zio.Chunk.empty); $selfRefIdent}"""
q"""{lazy val $selfRefIdent: _root_.zio.schema.Schema.Map[$keyType, $valueType] = _root_.zio.schema.Schema.Map.apply[$keyType, $valueType]($keySchema, $valueSchema, zio.Chunk.empty); $selfRefIdent}"""
}

recurse(tpe, List.empty[Frame[c.type]])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,9 @@ import zio.json.{
import zio.schema._
import zio.schema.annotation.optionalField
import zio.schema.codec.BinaryCodec._
import zio.schema.codec.DecodeError.ReadError
import zio.stream.ZPipeline
import zio.{ Chunk, ChunkBuilder, NonEmptyChunk, ZIO }
import zio.{ Cause, Chunk, ChunkBuilder, NonEmptyChunk, ZIO }

object JsonCodec extends BinaryCodec {

Expand All @@ -39,15 +40,15 @@ object JsonCodec extends BinaryCodec {
override def decoderFor[A](schema: Schema[A]): BinaryDecoder[A] =
new BinaryDecoder[A] {

override def decode(chunk: Chunk[Byte]): Either[String, A] =
override def decode(chunk: Chunk[Byte]): Either[DecodeError, A] =
JsonDecoder.decode(
schema,
new String(chunk.toArray, JsonEncoder.CHARSET)
)

override def streamDecoder: BinaryStreamDecoder[A] =
ZPipeline.fromChannel(
ZPipeline.utfDecode.channel.mapError(_.toString)
ZPipeline.utfDecode.channel.mapError(cce => ReadError(Cause.fail(cce), cce.getMessage))
) >>>
ZPipeline.groupAdjacentBy[String, Unit](_ => ()) >>>
ZPipeline.map[(Unit, NonEmptyChunk[String]), String] {
Expand Down Expand Up @@ -303,8 +304,11 @@ object JsonCodec extends BinaryCodec {
import Codecs._
import ProductDecoder._

final def decode[A](schema: Schema[A], json: String): Either[String, A] =
schemaDecoder(schema).decodeJson(json)
final def decode[A](schema: Schema[A], json: String): Either[DecodeError, A] =
schemaDecoder(schema).decodeJson(json) match {
case Left(value) => Left(ReadError(Cause.empty, value))
case Right(value) => Right(value)
}

//scalafmt: { maxColumn = 400, optIn.configStyleArguments = false }
private[codec] def schemaDecoder[A](schema: Schema[A]): ZJsonDecoder[A] = schema match {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import zio.json.{ DeriveJsonEncoder, JsonEncoder }
import zio.schema.CaseSet._
import zio.schema._
import zio.schema.annotation.optionalField
import zio.schema.codec.DecodeError.ReadError
import zio.schema.codec.JsonCodec.JsonEncoder.charSequenceToByteChunk
import zio.stream.ZStream
import zio.test.Assertion._
Expand Down Expand Up @@ -703,9 +704,9 @@ object JsonCodecSpec extends ZIOSpecDefault {
val stream = ZStream
.fromChunk(charSequenceToByteChunk(json))
.via(JsonCodec.decoder(schema))
.catchAll(ZStream.succeed[String](_))
.catchAll(ZStream.succeed[DecodeError](_))
.runHead
assertZIO(stream)(isSome(equalTo(JsonError.render(errors))))
assertZIO(stream)(isSome(equalTo(ReadError(Cause.empty, JsonError.render(errors)))))
}

private def assertDecodes[A](schema: Schema[A], value: A, chunk: Chunk[Byte]) = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4,9 +4,10 @@ import java.math.{ BigInteger, MathContext }
import java.time.{ Duration, Month, MonthDay, Period, Year, YearMonth }

import zio.schema.codec.BinaryCodec.{ BinaryDecoder, BinaryEncoder, BinaryStreamDecoder, BinaryStreamEncoder }
import zio.schema.codec.DecodeError.ReadError
import zio.schema.{ Schema, StandardType }
import zio.stream.ZPipeline
import zio.{ Chunk, ZIO }
import zio.{ Cause, Chunk, ZIO }

object MessagePackCodec extends BinaryCodec {
override def encoderFor[A](schema: Schema[A]): BinaryEncoder[A] = new BinaryEncoder[A] {
Expand All @@ -24,9 +25,9 @@ object MessagePackCodec extends BinaryCodec {

override def decoderFor[A](schema: Schema[A]): BinaryDecoder[A] = new BinaryDecoder[A] {

override def decode(chunk: Chunk[Byte]): Either[String, A] =
override def decode(chunk: Chunk[Byte]): Either[DecodeError, A] =
if (chunk.isEmpty)
Left("No bytes to decode")
Left(ReadError(Cause.empty, "No bytes to decode"))
else
decodeChunk(chunk)

Expand All @@ -41,9 +42,7 @@ object MessagePackCodec extends BinaryCodec {
new MessagePackDecoder(chunk)
.decode(schema)
.left
.map(
err => s"Error at path /${err.path.mkString(".")}: ${err.error}"
)
.map(identity)
}

//TODO those are duplicates from ThriftCodec
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import scala.util.{ Failure, Success, Try }

import org.msgpack.core.{ MessagePack, MessageUnpacker }

import zio.schema.codec.DecodeError.MalformedFieldWithPath
import zio.schema.codec.MessagePackDecoder._
import zio.schema.{ DynamicValueSchema, Schema, StandardType }
import zio.{ Chunk, ChunkBuilder }
Expand Down Expand Up @@ -104,7 +105,7 @@ private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) {
}

private def decodeTransform[A, B](path: Path, schema: Schema[B], f: B => scala.util.Either[String, A]): Result[A] =
decodeValue(path, schema).flatMap(a => f(a).left.map(msg => Error(path, msg)))
decodeValue(path, schema).flatMap(a => f(a).left.map(msg => MalformedFieldWithPath(path, msg)))

private def decodeRecord[Z](path: Path, fields: Seq[Schema.Field[Z, _]]): Result[ListMap[String, _]] =
decodeStructure(path, fields.map(f => f.name -> f.schema).toMap)
Expand Down Expand Up @@ -151,8 +152,8 @@ private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) {
(decodeValue(path, schema.keySchema), decodeValue(path, schema.valueSchema)) match {
case (Right(key), Right(value)) => decodeElements(n - 1, m += ((key, value)))
case (l, r) =>
val key = l.fold(_.error, _.toString)
val value = r.fold(_.error, _.toString)
val key = l.fold(_.message, _.toString)
val value = r.fold(_.message, _.toString)
fail(path, s"Error decoding Map element (key: $key; value: $value)")
}
} else {
Expand Down Expand Up @@ -608,12 +609,12 @@ private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) {

object MessagePackDecoder {
type Path = Chunk[String]
case class Error(path: Path, error: String)
type Result[A] = scala.util.Either[Error, A]

type Result[A] = scala.util.Either[DecodeError, A]

private def succeed[A](a: => A): Result[A] =
Right(a)

private def fail(path: Path, failure: String): Result[Nothing] =
Left(Error(path, failure))
Left(MalformedFieldWithPath(path, failure))
}
Original file line number Diff line number Diff line change
Expand Up @@ -697,7 +697,7 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
},
test("empty input by non streaming variant") {
assertZIO(decodeNS(Schema[Int], "").exit)(
fails(equalTo("No bytes to decode"))
failsWithA[DecodeError]
)
}
),
Expand All @@ -707,18 +707,10 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
d <- decode(Record.schemaRecord, "0F").exit
d2 <- decodeNS(Record.schemaRecord, "0F").exit
} yield assert(d)(
fails(
equalTo(
"Error at path /: Error reading object header: [org.msgpack.core.MessageTypeException: Expected Map, but got Integer (0f)]"
)
)
failsWithA[DecodeError]
) &&
assert(d2)(
fails(
equalTo(
"Error at path /: Error reading object header: [org.msgpack.core.MessageTypeException: Expected Map, but got Integer (0f)]"
)
)
failsWithA[DecodeError]
)
},
test("missing value") {
Expand All @@ -738,18 +730,10 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
}
d2 <- decode(Record.schemaRecord, bytes2).exit
} yield assert(d)(
fails(
equalTo(
"Error at path /fieldId:[value]: Cannot read int: [org.msgpack.core.MessageInsufficientBufferException]"
)
)
failsWithA[DecodeError]
) &&
assert(d2)(
fails(
equalTo(
"Error at path /: Error reading field name on index 2: [org.msgpack.core.MessageTypeException: Expected String, but got Integer (7b)]"
)
)
failsWithA[DecodeError]
)
},
test("unable to decode") {
Expand All @@ -763,11 +747,7 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
}
d <- decode(Record.schemaRecord, bytes).exit
} yield assert(d)(
fails(
equalTo(
"Error at path /fieldId:[value]: Cannot read int: [org.msgpack.core.MessageTypeException: Expected Integer, but got String (a9)]"
)
)
failsWithA[DecodeError]
)
},
test("unknown type") {
Expand All @@ -778,11 +758,7 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
}
d <- decode(Record.schemaRecord, bytes).exit
} yield assert(d)(
fails(
startsWithString(
"Error at path /: Could not find schema for field: [This is number one bullshit] on index: 1"
)
)
failsWithA[DecodeError]
)
}
)
Expand Down Expand Up @@ -1006,32 +982,32 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
def encodeNS[A](schema: Schema[A], input: A): ZIO[Any, Nothing, Chunk[Byte]] =
ZIO.succeed(MessagePackCodec.encode(schema)(input))

def decode[A](schema: Schema[A], hex: String): ZIO[Any, String, Chunk[A]] =
def decode[A](schema: Schema[A], hex: String): ZIO[Any, DecodeError, Chunk[A]] =
ZStream
.fromChunk(fromHex(hex))
.via(MessagePackCodec.decoder(schema))
.run(ZSink.collectAll)

//NS == non streaming variant of decode
def decodeNS[A](schema: Schema[A], hex: String): ZIO[Any, String, A] =
ZIO.succeed(MessagePackCodec.decode(schema)(fromHex(hex))).absolve[String, A]
def decodeNS[A](schema: Schema[A], hex: String): ZIO[Any, DecodeError, A] =
ZIO.succeed(MessagePackCodec.decode(schema)(fromHex(hex))).absolve[DecodeError, A]

def encodeAndDecode[A](schema: Schema[A], input: A): ZIO[Any, String, Chunk[A]] =
def encodeAndDecode[A](schema: Schema[A], input: A): ZIO[Any, DecodeError, Chunk[A]] =
ZStream
.succeed(input)
.via(MessagePackCodec.encoder(schema))
.via(MessagePackCodec.decoder(schema))
.run(ZSink.collectAll)

def encodeAndDecode[A](encodeSchema: Schema[A], decodeSchema: Schema[A], input: A): ZIO[Any, String, Chunk[A]] =
def encodeAndDecode[A](encodeSchema: Schema[A], decodeSchema: Schema[A], input: A): ZIO[Any, DecodeError, Chunk[A]] =
ZStream
.succeed(input)
.via(MessagePackCodec.encoder(encodeSchema))
.via(MessagePackCodec.decoder(decodeSchema))
.run(ZSink.collectAll)

//NS == non streaming variant of encodeAndDecode
def encodeAndDecodeNS[A](schema: Schema[A], input: A, print: Boolean = false): ZIO[Any, String, A] =
def encodeAndDecodeNS[A](schema: Schema[A], input: A, print: Boolean = false): ZIO[Any, DecodeError, A] =
ZIO
.succeed(input)
.tap(value => Console.printLine(s"Input Value: $value").when(print).ignore)
Expand All @@ -1040,7 +1016,7 @@ object MessagePackCodecSpec extends ZIOSpecDefault {
.map(ch => MessagePackCodec.decode(schema)(ch))
.absolve

def encodeAndDecodeNS[A](encodeSchema: Schema[A], decodeSchema: Schema[A], input: A): ZIO[Any, String, A] =
def encodeAndDecodeNS[A](encodeSchema: Schema[A], decodeSchema: Schema[A], input: A): ZIO[Any, DecodeError, A] =
ZIO
.succeed(input)
.map(a => MessagePackCodec.encode(encodeSchema)(a))
Expand Down
Loading

0 comments on commit 0e09815

Please sign in to comment.