From 6a0e9149864dbf5a612f79d4dd2d294159a47e04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pablo=20Femen=C3=ADa?= <131800808+pablf@users.noreply.github.com> Date: Sat, 20 Jan 2024 22:36:15 +0100 Subject: [PATCH] add schema fallback (#645) --- .../scala-2/zio/schema/DynamicValueGen.scala | 7 + .../test/scala-2/zio/schema/SchemaGen.scala | 27 ++++ .../scala/zio/schema/codec/AvroCodec.scala | 57 +++++++- .../zio/schema/codec/AvroSchemaCodec.scala | 4 + .../zio/schema/codec/AvroCodecSpec.scala | 49 ++++++- .../zio/schema/codec/BsonSchemaCodec.scala | 80 ++++++++++- .../main/scala/zio/schema/CachedDeriver.scala | 3 + .../scala/zio/schema/codec/JsonCodec.scala | 109 +++++++++++++++ .../main/scala/zio/schema/codec/package.scala | 1 + .../zio/schema/codec/JsonCodecSpec.scala | 126 ++++++++++++++++++ .../zio/schema/codec/MessagePackDecoder.scala | 47 +++++-- .../zio/schema/codec/MessagePackEncoder.scala | 20 ++- .../schema/codec/MessagePackCodecSpec.scala | 69 +++++++++- .../zio/schema/codec/ProtobufCodec.scala | 54 ++++++++ .../zio/schema/codec/ProtobufCodecSpec.scala | 67 ++++++++++ .../scala/zio/schema/codec/ThriftCodec.scala | 50 +++++++ .../zio/schema/codec/ThriftCodecSpec.scala | 67 ++++++++++ .../src/main/scala/zio/schema/Differ.scala | 8 ++ .../main/scala/zio/schema/DynamicValue.scala | 12 ++ .../src/main/scala/zio/schema/Fallback.scala | 63 +++++++++ .../MutableSchemaBasedValueBuilder.scala | 68 ++++++++++ .../MutableSchemaBasedValueProcessor.scala | 62 ++++++++- .../src/main/scala/zio/schema/Patch.scala | 43 ++++++ .../src/main/scala/zio/schema/Schema.scala | 77 +++++++++++ .../scala/zio/schema/SchemaEquality.scala | 4 + .../scala/zio/schema/meta/AstRenderer.scala | 20 +++ .../schema/meta/ExtensibleMetaSchema.scala | 70 ++++++++++ 27 files changed, 1245 insertions(+), 19 deletions(-) create mode 100644 zio-schema/shared/src/main/scala/zio/schema/Fallback.scala diff --git a/tests/shared/src/test/scala-2/zio/schema/DynamicValueGen.scala b/tests/shared/src/test/scala-2/zio/schema/DynamicValueGen.scala index 577dc87c3..1b84b6e5c 100644 --- a/tests/shared/src/test/scala-2/zio/schema/DynamicValueGen.scala +++ b/tests/shared/src/test/scala-2/zio/schema/DynamicValueGen.scala @@ -79,6 +79,8 @@ object DynamicValueGen { case Schema.Tuple2(left, right, _) => anyDynamicTupleValue(left, right) case Schema.Either(left, right, _) => Gen.oneOf(anyDynamicLeftValueOfSchema(left), anyDynamicRightValueOfSchema(right)) + case Schema.Fallback(left, right, _, _) => + Gen.oneOf(anyDynamicLeftValueOfSchema(left), anyDynamicRightValueOfSchema(right), anyDynamicBothValueOfSchema(left, right)) case Schema.Transform(schema, _, _, _, _) => anyDynamicValueOfSchema(schema) case Schema.Fail(message, _) => Gen.const(DynamicValue.Error(message)) case l @ Schema.Lazy(_) => anyDynamicValueOfSchema(l.schema) @@ -92,6 +94,11 @@ object DynamicValueGen { def anyDynamicRightValueOfSchema[A](schema: Schema[A]): Gen[Sized, DynamicValue.RightValue] = anyDynamicValueOfSchema(schema).map(DynamicValue.RightValue(_)) + def anyDynamicBothValueOfSchema[A, B](left: Schema[A], right: Schema[A]): Gen[Sized, DynamicValue.BothValue] = + anyDynamicValueOfSchema(left).zip(anyDynamicValueOfSchema(right)).map { + case (l, r) => DynamicValue.BothValue(l, r) + } + def anyDynamicSomeValueOfSchema[A](schema: Schema[A]): Gen[Sized, DynamicValue.SomeValue] = anyDynamicValueOfSchema(schema).map(DynamicValue.SomeValue(_)) diff --git a/tests/shared/src/test/scala-2/zio/schema/SchemaGen.scala b/tests/shared/src/test/scala-2/zio/schema/SchemaGen.scala index d4519a3e3..5fba0879e 100644 --- a/tests/shared/src/test/scala-2/zio/schema/SchemaGen.scala +++ b/tests/shared/src/test/scala-2/zio/schema/SchemaGen.scala @@ -120,6 +120,33 @@ object SchemaGen { value <- gen } yield (schema, value) + def anyFallback(fullDecode: Boolean): Gen[Sized, Schema.Fallback[_, _]] = + for { + left <- anyPrimitive + right <- anyPrimitive + } yield Schema.Fallback(left, right, fullDecode) + + type FallbackAndGen[A, B] = (Schema.Fallback[A, B], Gen[Sized, Fallback[A, B]]) + + def anyFallbackAndGen(fullDecode: Boolean): Gen[Sized, FallbackAndGen[_, _]] = + for { + (leftSchema, leftGen) <- anyPrimitiveAndGen + (rightSchema, rightGen) <- anyPrimitiveAndGen + } yield ( + Schema.Fallback(leftSchema, rightSchema, fullDecode), + Gen.oneOf(leftGen.map(Fallback.Left(_)), rightGen.map(Fallback.Right(_)), leftGen.zip(rightGen).map { + case (l, r) => Fallback.Both(l, r) + }) + ) + + type FallbackAndValue[A, B] = (Schema.Fallback[A, B], Fallback[A, B]) + + def anyFallbackAndValue(fullDecode: Boolean): Gen[Sized, FallbackAndValue[_, _]] = + for { + (schema, gen) <- anyFallbackAndGen(fullDecode) + value <- gen + } yield (schema, value) + lazy val anyTuple: Gen[Sized, Schema.Tuple2[_, _]] = anySchema.zipWith(anySchema) { (a, b) => Schema.Tuple2(a, b) diff --git a/zio-schema-avro/src/main/scala/zio/schema/codec/AvroCodec.scala b/zio-schema-avro/src/main/scala/zio/schema/codec/AvroCodec.scala index 4359d0a21..0d7677d61 100644 --- a/zio-schema-avro/src/main/scala/zio/schema/codec/AvroCodec.scala +++ b/zio-schema-avro/src/main/scala/zio/schema/codec/AvroCodec.scala @@ -19,7 +19,7 @@ import org.apache.avro.io.{ DecoderFactory, EncoderFactory } import org.apache.avro.util.Utf8 import org.apache.avro.{ Conversions, LogicalTypes, Schema => SchemaAvro } -import zio.schema.{ FieldSet, Schema, StandardType, TypeId } +import zio.schema.{ Fallback, FieldSet, Schema, StandardType, TypeId } import zio.stream.ZPipeline import zio.{ Chunk, Unsafe, ZIO } @@ -213,6 +213,7 @@ object AvroCodec { case Schema.Fail(message, _) => Left(DecodeError.MalformedFieldWithPath(Chunk.empty, message)) case Schema.Tuple2(left, right, _) => decodeTuple2(raw, left, right).map(_.asInstanceOf[A]) case Schema.Either(left, right, _) => decodeEitherValue(raw, left, right) + case s @ Schema.Fallback(_, _, _, _) => decodeFallbackValue(raw, s) case lzy @ Schema.Lazy(_) => decodeValue(raw, lzy.schema) case unknown => Left(DecodeError.MalformedFieldWithPath(Chunk.empty, s"Unknown schema: $unknown")) } @@ -468,6 +469,7 @@ object AvroCodec { val result2 = decodeValue(record.get("_2"), schemaRight) result1.flatMap(a => result2.map(b => (a, b))) } + private def decodeEitherValue[A, B](value: Any, schemaLeft: Schema[A], schemaRight: Schema[B]) = { val record = value.asInstanceOf[GenericRecord] val result = decodeValue(record.get("value"), schemaLeft) @@ -475,6 +477,34 @@ object AvroCodec { else decodeValue(record.get("value"), schemaRight).map(Right(_)) } + private def decodeFallbackValue[A, B](value: Any, schema: Schema.Fallback[A, B]) = { + var error: Option[DecodeError] = None + + val record = value.asInstanceOf[GenericRecord] + val left: Option[A] = decodeValue(record.get("_1"), Schema.Optional(schema.left)) match { + case Right(value) => value + case Left(err) => { + error = Some(err) + None + } + } + + val right = left match { + case Some(_) => + if (schema.fullDecode) decodeValue(record.get("_2"), Schema.Optional(schema.right)).getOrElse(None) + else None + case _ => + decodeValue(record.get("_2"), Schema.Optional(schema.right)).getOrElse(None) + } + + (left, right) match { + case (Some(a), Some(b)) => Right(Fallback.Both(a, b)) + case (_, Some(b)) => Right(Fallback.Right(b)) + case (Some(a), _) => Right(Fallback.Left(a)) + case _ => Left(error.get) + } + } + private def decodeOptionalValue[A](value: Any, schema: Schema[A]) = if (value == null) Right(None) else decodeValue(value, schema).map(Some(_)) @@ -631,8 +661,9 @@ object AvroCodec { case Schema.Optional(schema, _) => encodeOption(schema, a) case Schema.Tuple2(left, right, _) => encodeTuple2(left.asInstanceOf[Schema[Any]], right.asInstanceOf[Schema[Any]], a) - case Schema.Either(left, right, _) => encodeEither(left, right, a) - case Schema.Lazy(schema0) => encodeValue(a, schema0()) + case Schema.Either(left, right, _) => encodeEither(left, right, a) + case s @ Schema.Fallback(_, _, _, _) => encodeFallback(s, a) + case Schema.Lazy(schema0) => encodeValue(a, schema0()) case Schema.CaseClass0(_, _, _) => encodeCaseClass(schema, a, Seq.empty: _*) //encodePrimitive((), StandardType.UnitType) case Schema.CaseClass1(_, f, _, _) => encodeCaseClass(schema, a, f) @@ -942,6 +973,26 @@ object AvroCodec { result.build() } + private def encodeFallback[A, B](s: Schema.Fallback[A, B], f: zio.schema.Fallback[A, B]): Any = { + val schema = AvroSchemaCodec + .encodeToApacheAvro(s) + .getOrElse(throw new Exception("Avro schema could not be generated for Fallback.")) + + val value: (Option[A], Option[B]) = f match { + case zio.schema.Fallback.Left(a) => (Some(a), None) + case zio.schema.Fallback.Right(b) => (None, Some(b)) + case zio.schema.Fallback.Both(a, b) => (Some(a), Some(b)) + } + + val left = encodeOption[A](s.left, value._1) + val right = encodeOption[B](s.right, value._2) + + val record = new GenericData.Record(schema) + record.put("_1", left) + record.put("_2", right) + record + } + private def encodeTuple2[A](schema1: Schema[Any], schema2: Schema[Any], a: A) = { val schema = AvroSchemaCodec .encodeToApacheAvro(Schema.Tuple2(schema1, schema2, Chunk.empty)) diff --git a/zio-schema-avro/src/main/scala/zio/schema/codec/AvroSchemaCodec.scala b/zio-schema-avro/src/main/scala/zio/schema/codec/AvroSchemaCodec.scala index bb64aeac7..cad31b7d1 100644 --- a/zio-schema-avro/src/main/scala/zio/schema/codec/AvroSchemaCodec.scala +++ b/zio-schema-avro/src/main/scala/zio/schema/codec/AvroSchemaCodec.scala @@ -415,6 +415,9 @@ object AvroSchemaCodec extends AvroSchemaCodec { name <- getName(e) } yield wrapAvro(union, name, EitherWrapper) + case Schema.Fallback(left, right, _, _) => + toAvroSchema(Schema.Tuple2(Schema.Optional(left), Schema.Optional(right))) + case Lazy(schema0) => toAvroSchema(schema0()) case Dynamic(_) => toAvroSchema(Schema[MetaSchema]) } @@ -784,6 +787,7 @@ object AvroSchemaCodec extends AvroSchemaCodec { case _ => Left("ZIO schema wrapped either must have exactly two cases") } case e: Schema.Either[_, _] => Right(e) + case f: Schema.Fallback[_, _] => Right(f) case c: CaseClass0[_] => Right(c) case c: CaseClass1[_, _] => Right(c) case c: CaseClass2[_, _, _] => Right(c) diff --git a/zio-schema-avro/src/test/scala-2/zio/schema/codec/AvroCodecSpec.scala b/zio-schema-avro/src/test/scala-2/zio/schema/codec/AvroCodecSpec.scala index 8f1526aaf..3d8bf3ce3 100644 --- a/zio-schema-avro/src/test/scala-2/zio/schema/codec/AvroCodecSpec.scala +++ b/zio-schema-avro/src/test/scala-2/zio/schema/codec/AvroCodecSpec.scala @@ -24,7 +24,7 @@ import org.apache.avro.generic.GenericData import zio._ import zio.schema.codec.AvroAnnotations.avroEnum -import zio.schema.{ DeriveSchema, Schema } +import zio.schema.{ DeriveSchema, Fallback, Schema } import zio.stream.ZStream import zio.test._ @@ -131,6 +131,7 @@ object AvroCodecSpec extends ZIOSpecDefault { collectionsEncoderSpec, optionEncoderSpec, eitherEncoderSpec, + fallbackEncoderSpec, tupleEncoderSpec, genericRecordEncoderSpec, caseClassEncoderSpec, @@ -138,6 +139,7 @@ object AvroCodecSpec extends ZIOSpecDefault { primitiveDecoderSpec, optionDecoderSpec, eitherDecoderSpec, + fallbackDecoderSpec, tupleDecoderSpec, sequenceDecoderSpec, genericRecordDecoderSpec, @@ -308,6 +310,24 @@ object AvroCodecSpec extends ZIOSpecDefault { } ) + private val fallbackEncoderSpec = suite("Avro Codec - Encoder Fallback spec")( + test("Encode Fallback.Right") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[Int, String]] + val bytes = codec.encode(Fallback.Right("John")) + assertTrue(bytes.length == 7) + }, + test("Encode Fallback.Left") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[Int, String]] + val bytes = codec.encode(Fallback.Left(42)) + assertTrue(bytes.length == 3) + }, + test("Encode Fallback.Both") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[Int, String]] + val bytes = codec.encode(Fallback.Both(42, "John")) + assertTrue(bytes.length == 8) + } + ) + private val tupleEncoderSpec = suite("Avro Codec - Encode Tuples spec")( test("Encode Tuple2[Int, String]") { val codec = AvroCodec.schemaBasedBinaryCodec[(Int, String)] @@ -590,6 +610,33 @@ object AvroCodecSpec extends ZIOSpecDefault { } ) + private val fallbackDecoderSpec = suite("Avro Codec - Fallback Decoder spec")( + test("Decode Fallback") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[String, Int]] + val bytes = codec.encode(Fallback.Right(42)) + val result = codec.decode(bytes) + assertTrue(result == Right(Fallback.Right(42))) + }, + test("Decode Fallback[List[String], Int]") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[List[String], Int]] + val bytes = codec.encode(Fallback.Left(List("John", "Adam", "Daniel"))) + val result = codec.decode(bytes) + assertTrue(result == Right(Fallback.Left(List("John", "Adam", "Daniel")))) + }, + test("Decode Fallback.Both full decode") { + val codec = AvroCodec.schemaBasedBinaryCodec(Schema.Fallback[String, Int](Schema[String], Schema[Int], true)) + val bytes = codec.encode(Fallback.Both("hello", 42)) + val result = codec.decode(bytes) + assertTrue(result == Right(Fallback.Both("hello", 42))) + }, + test("Decode Fallback.Both non full decode") { + val codec = AvroCodec.schemaBasedBinaryCodec[Fallback[String, Int]] + val bytes = codec.encode(Fallback.Both("hello", 42)) + val result = codec.decode(bytes) + assertTrue(result == Right(Fallback.Left("hello"))) + } + ) + private val tupleDecoderSpec = suite("Avro Codec - Tuple Decoder Spec")( test("Decode Tuple2") { val codec = AvroCodec.schemaBasedBinaryCodec[(Int, String)] diff --git a/zio-schema-bson/src/main/scala/zio/schema/codec/BsonSchemaCodec.scala b/zio-schema-bson/src/main/scala/zio/schema/codec/BsonSchemaCodec.scala index 4e57c31fe..fcfd78ac9 100644 --- a/zio-schema-bson/src/main/scala/zio/schema/codec/BsonSchemaCodec.scala +++ b/zio-schema-bson/src/main/scala/zio/schema/codec/BsonSchemaCodec.scala @@ -37,7 +37,7 @@ import zio.schema.annotation.{ transientCase, transientField } -import zio.schema.{ DynamicValue, Schema, StandardType, TypeId } +import zio.schema.{ DynamicValue, Fallback, Schema, StandardType, TypeId } import zio.{ Chunk, ChunkBuilder, Unsafe } object BsonSchemaCodec { @@ -312,6 +312,78 @@ object BsonSchemaCodec { } } + protected[codec] def fallbackEncoder[A: BsonEncoder, B: BsonEncoder]: BsonEncoder[Fallback[A, B]] = + new BsonEncoder[Fallback[A, B]] { + override def encode(writer: BsonWriter, value: Fallback[A, B], ctx: BsonEncoder.EncoderContext): Unit = { + val nextCtx = BsonEncoder.EncoderContext.default + + if (!ctx.inlineNextObject) writer.writeStartDocument() + + value match { + case Fallback.Left(value) => + BsonEncoder[A].encode(writer, value, nextCtx) + case Fallback.Right(value) => + BsonEncoder[B].encode(writer, value, nextCtx) + case Fallback.Both(left, right) => + writer.writeStartArray() + BsonEncoder[A].encode(writer, left, nextCtx) + BsonEncoder[B].encode(writer, right, nextCtx) + writer.writeEndArray() + } + + if (!ctx.inlineNextObject) writer.writeEndDocument() + } + + override def toBsonValue(value: Fallback[A, B]): BsonValue = value match { + case Fallback.Left(value) => array(value.toBsonValue) + case Fallback.Right(value) => array(value.toBsonValue) + case Fallback.Both(left, right) => array(left.toBsonValue, right.toBsonValue) + } + } + + protected[codec] def fallbackDecoder[A: BsonDecoder, B: BsonDecoder]: BsonDecoder[Fallback[A, B]] = + new BsonDecoder[Fallback[A, B]] { + + override def decodeUnsafe( + reader: BsonReader, + trace: List[BsonTrace], + ctx: BsonDecoder.BsonDecoderContext + ): Fallback[A, B] = unsafeCall(trace) { + val nextCtx = BsonDecoder.BsonDecoderContext.default + + try { + Fallback.Left(BsonDecoder[A].decodeUnsafe(reader, trace, nextCtx)) + } catch { + case _: BsonDecoder.Error => + try { + Fallback.Right(BsonDecoder[B].decodeUnsafe(reader, trace, nextCtx)) + } catch { + case _: BsonDecoder.Error => throw BsonDecoder.Error(trace, "Both `left` and `right` cases missing.") + } + } + } + + override def fromBsonValueUnsafe( + value: BsonValue, + trace: List[BsonTrace], + ctx: BsonDecoder.BsonDecoderContext + ): Fallback[A, B] = + assumeType(trace)(BsonType.DOCUMENT, value) { value => + val nextCtx = BsonDecoder.BsonDecoderContext.default + + try { + Fallback.Left(BsonDecoder[A].fromBsonValueUnsafe(value, trace, nextCtx)) + } catch { + case _: BsonDecoder.Error => + try { + Fallback.Right(BsonDecoder[B].fromBsonValueUnsafe(value, trace, nextCtx)) + } catch { + case _: BsonDecoder.Error => throw BsonDecoder.Error(trace, "Both `left` and `right` cases missing.") + } + } + } + } + protected[codec] def failDecoder[A](message: String): BsonDecoder[A] = new BsonDecoder[A] { override def decodeUnsafe(reader: BsonReader, trace: List[BsonTrace], ctx: BsonDecoder.BsonDecoderContext): A = @@ -394,6 +466,7 @@ object BsonSchemaCodec { case Schema.Fail(_, _) => unitEncoder.contramap(_ => ()) case Schema.GenericRecord(_, structure, _) => genericRecordEncoder(structure.toChunk) case Schema.Either(left, right, _) => eitherEncoder(schemaEncoder(left), schemaEncoder(right)) + case Schema.Fallback(left, right, _, _) => fallbackEncoder(schemaEncoder(left), schemaEncoder(right)) case l @ Schema.Lazy(_) => schemaEncoder(l.schema) case r: Schema.Record[A] => caseClassEncoder(r) case e: Schema.Enum[A] => enumEncoder(e, e.cases) @@ -464,6 +537,8 @@ object BsonSchemaCodec { throw new Exception(s"DynamicValue.LeftValue is not supported in directDynamicMapping mode") case DynamicValue.RightValue(_) => throw new Exception(s"DynamicValue.RightValue is not supported in directDynamicMapping mode") + case DynamicValue.BothValue(_, _) => + throw new Exception(s"DynamicValue.BothValue is not supported in directDynamicMapping mode") case DynamicValue.DynamicAst(_) => throw new Exception(s"DynamicValue.DynamicAst is not supported in directDynamicMapping mode") case DynamicValue.Error(message) => @@ -496,6 +571,8 @@ object BsonSchemaCodec { throw new Exception(s"DynamicValue.LeftValue is not supported in directDynamicMapping mode") case DynamicValue.RightValue(_) => throw new Exception(s"DynamicValue.RightValue is not supported in directDynamicMapping mode") + case DynamicValue.BothValue(_, _) => + throw new Exception(s"DynamicValue.BothValue is not supported in directDynamicMapping mode") case DynamicValue.DynamicAst(_) => throw new Exception(s"DynamicValue.DynamicAst is not supported in directDynamicMapping mode") case DynamicValue.Error(message) => @@ -696,6 +773,7 @@ object BsonSchemaCodec { case Schema.Fail(message, _) => failDecoder(message) case Schema.GenericRecord(_, structure, _) => recordDecoder(structure.toChunk) case Schema.Either(left, right, _) => eitherDecoder(schemaDecoder(left), schemaDecoder(right)) + case Schema.Fallback(left, right, _, _) => fallbackDecoder(schemaDecoder(left), schemaDecoder(right)) case l @ Schema.Lazy(_) => schemaDecoder(l.schema) case s: Schema.Record[A] => caseClassDecoder(s) case e: Schema.Enum[A] => enumDecoder(e) diff --git a/zio-schema-derivation/shared/src/main/scala/zio/schema/CachedDeriver.scala b/zio-schema-derivation/shared/src/main/scala/zio/schema/CachedDeriver.scala index 5ed934417..53d516a65 100644 --- a/zio-schema-derivation/shared/src/main/scala/zio/schema/CachedDeriver.scala +++ b/zio-schema-derivation/shared/src/main/scala/zio/schema/CachedDeriver.scala @@ -122,6 +122,7 @@ private[schema] object CachedDeriver { final case class WithIdentityObject[A](inner: CacheKey[_], id: Any) extends CacheKey[A] final case class Optional[A](key: CacheKey[A]) extends CacheKey[A] final case class Either[A, B](leftKey: CacheKey[A], rightKey: CacheKey[B]) extends CacheKey[Either[A, B]] + final case class Fallback[A, B](leftKey: CacheKey[A], rightKey: CacheKey[B]) extends CacheKey[Fallback[A, B]] final case class Tuple2[A, B](leftKey: CacheKey[A], rightKey: CacheKey[B]) extends CacheKey[(A, B)] final case class Set[A](element: CacheKey[A]) extends CacheKey[Set[A]] final case class Map[K, V](key: CacheKey[K], valuew: CacheKey[V]) extends CacheKey[Map[K, V]] @@ -144,6 +145,8 @@ private[schema] object CachedDeriver { Tuple2(fromSchema(tuple.left), fromSchema(tuple.right)).asInstanceOf[CacheKey[A]] case either: Schema.Either[_, _] => Either(fromSchema(either.leftSchema), fromSchema(either.rightSchema)).asInstanceOf[CacheKey[A]] + case fallback: Schema.Fallback[_, _] => + Fallback(fromSchema(fallback.left), fromSchema(fallback.right)).asInstanceOf[CacheKey[A]] case Schema.Lazy(schema0) => fromSchema(schema0()) case Schema.Dynamic(_) => Misc(schema) case Schema.Fail(_, _) => Misc(schema) diff --git a/zio-schema-json/shared/src/main/scala/zio/schema/codec/JsonCodec.scala b/zio-schema-json/shared/src/main/scala/zio/schema/codec/JsonCodec.scala index 813aec6b4..823d4bce5 100644 --- a/zio-schema-json/shared/src/main/scala/zio/schema/codec/JsonCodec.scala +++ b/zio-schema-json/shared/src/main/scala/zio/schema/codec/JsonCodec.scala @@ -194,6 +194,7 @@ object JsonCodec { case Schema.Fail(_, _) => unitEncoder.contramap(_ => ()) case Schema.GenericRecord(_, structure, _) => recordEncoder(structure.toChunk, cfg) case Schema.Either(left, right, _) => ZJsonEncoder.either(schemaEncoder(left, cfg, discriminatorTuple), schemaEncoder(right, cfg, discriminatorTuple)) + case Schema.Fallback(left, right, _, _) => fallbackEncoder(schemaEncoder(left, cfg, discriminatorTuple), schemaEncoder(right, cfg, discriminatorTuple)) case l @ Schema.Lazy(_) => schemaEncoder(l.schema, cfg, discriminatorTuple) case Schema.CaseClass0(_, _, _) => caseClassEncoder(schema, discriminatorTuple, cfg) case Schema.CaseClass1(_, f, _, _) => caseClassEncoder(schema, discriminatorTuple, cfg, f) @@ -383,6 +384,16 @@ object JsonCodec { throw new Exception(s"DynamicValue.LeftValue is not supported in directDynamicMapping mode") case DynamicValue.RightValue(_) => throw new Exception(s"DynamicValue.RightValue is not supported in directDynamicMapping mode") + case DynamicValue.BothValue(left, right) => + out.write('[') + val indent_ = bump(indent) + pad(indent_, out) + directEncoder.unsafeEncode(left, indent_, out) + out.write(',') + if (indent.isDefined) ZJsonEncoder.pad(indent_, out) + directEncoder.unsafeEncode(right, indent_, out) + pad(indent, out) + out.write(']') case DynamicValue.DynamicAst(_) => throw new Exception(s"DynamicValue.DynamicAst is not supported in directDynamicMapping mode") case DynamicValue.Error(message) => @@ -472,6 +483,25 @@ object JsonCodec { } } + private def fallbackEncoder[A, B](left: ZJsonEncoder[A], right: ZJsonEncoder[B]): ZJsonEncoder[Fallback[A, B]] = + new ZJsonEncoder[Fallback[A, B]] { + + def unsafeEncode(f: Fallback[A, B], indent: Option[Int], out: Write): Unit = + f match { + case Fallback.Left(a) => left.unsafeEncode(a, indent, out) + case Fallback.Right(b) => right.unsafeEncode(b, indent, out) + case Fallback.Both(a, b) => + out.write('[') + if (indent.isDefined) pad(bump(indent), out) + left.unsafeEncode(a, indent, out) + out.write(',') + if (indent.isDefined) pad(bump(indent), out) + right.unsafeEncode(b, indent, out) + if (indent.isDefined) pad(indent, out) + out.write(']') + } + } + private def recordEncoder[Z](structure: Seq[Schema.Field[Z, _]], cfg: Config): ZJsonEncoder[ListMap[String, _]] = { (value: ListMap[String, _], indent: Option[Int], out: Write) => { @@ -531,6 +561,7 @@ object JsonCodec { case Schema.Fail(message, _) => failDecoder(message) case Schema.GenericRecord(_, structure, _) => recordDecoder(structure.toChunk) case Schema.Either(left, right, _) => ZJsonDecoder.either(schemaDecoder(left, -1), schemaDecoder(right, -1)) + case s @ Schema.Fallback(_, _, _, _) => fallbackDecoder(s) case l @ Schema.Lazy(_) => schemaDecoder(l.schema, discriminator) //case Schema.Meta(_, _) => astDecoder case s @ Schema.CaseClass0(_, _, _) => caseClass0Decoder(discriminator, s) @@ -809,6 +840,84 @@ object JsonCodec { (ListMap.newBuilder[String, Any] ++= builder.result()).result() } } + + private def fallbackDecoder[A, B](schema: Schema.Fallback[A, B]): ZJsonDecoder[Fallback[A, B]] = + new ZJsonDecoder[Fallback[A, B]] { + + def unsafeDecode(trace: List[JsonError], in: RetractReader): Fallback[A, B] = { + var left: Option[A] = None + var right: Option[B] = None + + case class BadEnd() extends Throwable + + try { + // If this doesn't throw exception, it is an array, so it encodes a `Fallback.Both` + zio.json.internal.Lexer.char(trace, in, '[') + + // get left element + if (Lexer.firstArrayElement(in)) { + val trace_ = JsonError.ArrayAccess(0) :: trace + try left = Some(schemaDecoder(schema.left).unsafeDecode(trace_, in)) + catch { + case _: UnsafeJson => () + } + + // read until ',' if left wasn't decoded + var continue = false + while (!continue) { + try { + Lexer.nextArrayElement(trace, in) + continue = true + } catch { + case _: UnsafeJson => () + } + } + } + + // get right element + if ((left == None || schema.fullDecode) && Lexer.firstArrayElement(in)) { + val trace_ = JsonError.ArrayAccess(1) :: trace + + try right = Some(schemaDecoder(schema.right).unsafeDecode(trace_, in)) + catch { + case _: UnsafeJson => () + } + + try Lexer.nextArrayElement(trace, in) + catch { + case _: UnsafeJson => throw BadEnd() + } + } + + } catch { + // It's not an array, so it is of type A or B + case BadEnd() => () + case _: UnsafeJson => { + in.retract() + val in2 = new zio.json.internal.WithRecordingReader(in, 64) + try { + left = Some(schemaDecoder(schema.left).unsafeDecode(trace, in2)) + } catch { + case UnsafeJson(_) => + in2.rewind() + right = Some(schemaDecoder(schema.right).unsafeDecode(trace, in2)) + } + } + } + + (left, right) match { + case (Some(a), Some(b)) => Fallback.Both(a, b) + case (Some(a), _) => Fallback.Left(a) + case (_, Some(b)) => Fallback.Right(b) + case _ => + throw UnsafeJson( + JsonError.Message("Fallback decoder was unable to decode both left and right sides") :: trace + ) + } + + } + } + } //scalafmt: { maxColumn = 400, optIn.configStyleArguments = false } diff --git a/zio-schema-json/shared/src/main/scala/zio/schema/codec/package.scala b/zio-schema-json/shared/src/main/scala/zio/schema/codec/package.scala index e0e3b781d..3a72e3031 100644 --- a/zio-schema-json/shared/src/main/scala/zio/schema/codec/package.scala +++ b/zio-schema-json/shared/src/main/scala/zio/schema/codec/package.scala @@ -65,6 +65,7 @@ package object json { case DynamicValue.Tuple(left, right) => Json.Arr(Chunk(toJson(left), toJson(right))) case DynamicValue.LeftValue(value) => Json.Obj("Left" -> toJson(value)) case DynamicValue.RightValue(value) => Json.Obj("Right" -> toJson(value)) + case DynamicValue.BothValue(_, _) => throw new Exception("DynamicValue.BothValue is unsupported") case DynamicValue.DynamicAst(_) => throw new Exception("DynamicValue.DynamicAst is unsupported") case DynamicValue.Error(_) => throw new Exception("DynamicValue.Error is unsupported") } diff --git a/zio-schema-json/shared/src/test/scala-2/zio/schema/codec/JsonCodecSpec.scala b/zio-schema-json/shared/src/test/scala-2/zio/schema/codec/JsonCodecSpec.scala index c228c49c7..60a5091f6 100644 --- a/zio-schema-json/shared/src/test/scala-2/zio/schema/codec/JsonCodecSpec.scala +++ b/zio-schema-json/shared/src/test/scala-2/zio/schema/codec/JsonCodecSpec.scala @@ -48,6 +48,29 @@ object JsonCodecSpec extends ZIOSpecDefault { assertEncodesJson(Schema.Primitive(StandardType.ZoneIdType), ZoneId.systemDefault()) } ), + suite("fallback")( + test("left") { + assertEncodesJson( + Schema.Fallback(Schema[Int], Schema[String]), + Fallback.Left(3), + "3" + ) + }, + test("right") { + assertEncodesJson( + Schema.Fallback(Schema[Int], Schema[String]), + Fallback.Right("hello"), + """"hello"""" + ) + }, + test("both") { + assertEncodesJson( + Schema.Fallback(Schema[Int], Schema[String]), + Fallback.Both(3, "hello"), + """[3,"hello"]""" + ) + } + ), suite("optional")( test("of primitives") { assertEncodesJson( @@ -423,6 +446,43 @@ object JsonCodecSpec extends ZIOSpecDefault { } } ), + suite("fallback")( + test("correctly fallbacks to right") { + assertDecodes( + Schema.Fallback(Schema[Int], Schema[String]), + Fallback.Right("hello"), + charSequenceToByteChunk("""["wrong","hello"]""") + ) + }, + test("correctly fallbacks to left") { + assertDecodes( + Schema.Fallback(Schema[Int], Schema[String]), + Fallback.Left(30), + charSequenceToByteChunk("""[30,"hello"]""") + ) + }, + test("correctly fallbacks to right with full decode") { + assertDecodes( + Schema.Fallback(Schema[Int], Schema[String], true), + Fallback.Right("hello"), + charSequenceToByteChunk("""[ "wrong", "hello"]""") + ) + }, + test("correctly fallbacks to both with full decode") { + assertDecodes( + Schema.Fallback(Schema[Int], Schema[String], true), + Fallback.Both(30, "hello"), + charSequenceToByteChunk("""[ 30, "hello"]""") + ) + }, + test("correctly fallbacks to left with full decode") { + assertDecodes( + Schema.Fallback(Schema[Int], Schema[String], true), + Fallback.Left(30), + charSequenceToByteChunk("""[30,30]""") + ) + } + ), suite("case class")( test("case object") { assertDecodes(schemaObject, Singleton, charSequenceToByteChunk("{}")) @@ -834,6 +894,42 @@ object JsonCodecSpec extends ZIOSpecDefault { } } ), + suite("fallback")( + test("of enums") { + check(for { + (left, value) <- SchemaGen.anyEnumerationAndValue + (right, _) <- SchemaGen.anyEnumerationAndValue + } yield (Schema.Fallback(left, right), Fallback.Left(value))) { + case (schema, value) => assertEncodesThenDecodesFallback(schema, value) + } + }, + test("of map") { + check( + for { + left <- SchemaGen.anyMapAndValue + right <- SchemaGen.anyMapAndValue + } yield ( + Schema + .Fallback(left._1.asInstanceOf[Schema[Map[Any, Any]]], right._1.asInstanceOf[Schema[Map[Any, Any]]]), + Fallback.Left(left._2) + ) + ) { + case (schema, value) => + assertEncodesThenDecodesFallback[Map[Any, Any], Map[Any, Any]]( + schema, + value.asInstanceOf[Fallback[Map[Any, Any], Map[Any, Any]]] + ) + } + }, + test("of records") { + check(for { + (left, a) <- SchemaGen.anyRecordAndValue() + primitiveSchema <- SchemaGen.anyPrimitive + } yield (Schema.Fallback(left, primitiveSchema), Fallback.Left(a))) { + case (schema, value) => assertEncodesThenDecodesFallback(schema, value) + } + } + ), suite("optional")( test("of primitive") { check(SchemaGen.anyOptionalAndValue) { @@ -1313,6 +1409,36 @@ object JsonCodecSpec extends ZIOSpecDefault { assertZIO(result)(equalTo(Chunk(value))) } + private def assertEncodesThenDecodesFallback[A, B]( + schema: Schema.Fallback[A, B], + value: Fallback[A, B] + ): ZIO[Any, Nothing, TestResult] = + ZStream + .succeed(value) + .via(JsonCodec.schemaBasedBinaryCodec[zio.schema.Fallback[A, B]](JsonCodec.Config.default)(schema).streamEncoder) + .runCollect + .flatMap { encoded => + ZStream + .fromChunk(encoded) + .via( + JsonCodec.schemaBasedBinaryCodec[zio.schema.Fallback[A, B]](JsonCodec.Config.default)(schema).streamDecoder + ) + .runCollect + } + .either + .flatMap { result => + val expected = if (schema.fullDecode) value else value.simplify + result.map(_.headOption.getOrElse(expected)) match { + case Right(obtained) => + if (expected == obtained) + ZIO.succeed(assertTrue(expected == obtained)) + else + assertEncodesThenDecodesFallback(schema, obtained) + case Left(_) => ZIO.succeed(assertTrue(false)) + } + + } + private def assertEncodesThenDecodes[A](schema: Schema[A], value: A, print: Boolean = false) = assertEncodesThenDecodesWithDifferentSchemas(schema, schema, value, (x: A, y: A) => x == y, print) diff --git a/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackDecoder.scala b/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackDecoder.scala index 821baf4b6..0e7e3bc05 100644 --- a/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackDecoder.scala +++ b/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackDecoder.scala @@ -12,7 +12,7 @@ import org.msgpack.core.{ MessagePack, MessageUnpacker } import zio.schema.codec.DecodeError.MalformedFieldWithPath import zio.schema.codec.MessagePackDecoder._ -import zio.schema.{ DynamicValue, Schema, StandardType } +import zio.schema.{ DynamicValue, Fallback, Schema, StandardType } import zio.{ Chunk, ChunkBuilder } private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) { @@ -27,16 +27,17 @@ private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) { case Schema.GenericRecord(_, structure, _) => val fields = structure.toChunk decodeRecord(path, fields) - case seqSchema @ Schema.Sequence(_, _, _, _, _) => decodeSequence(path, seqSchema) - case mapSchema @ Schema.Map(_, _, _) => decodeMap(path, mapSchema) - case setSchema @ Schema.Set(_, _) => decodeSet(path, setSchema) - case Schema.Transform(schema, f, _, _, _) => decodeTransform(path, schema, f) - case Schema.Primitive(standardType, _) => decodePrimitive(path, standardType) - case Schema.Tuple2(left, right, _) => decodeTuple(path, left, right) - case optionalSchema @ Schema.Optional(_, _) => decodeOptional(path, optionalSchema) - case Schema.Fail(message, _) => fail(path, message) - case Schema.Either(left, right, _) => decodeEither(path, left, right) - case lzy @ Schema.Lazy(_) => decodeValue(path, lzy.schema) + case seqSchema @ Schema.Sequence(_, _, _, _, _) => decodeSequence(path, seqSchema) + case mapSchema @ Schema.Map(_, _, _) => decodeMap(path, mapSchema) + case setSchema @ Schema.Set(_, _) => decodeSet(path, setSchema) + case Schema.Transform(schema, f, _, _, _) => decodeTransform(path, schema, f) + case Schema.Primitive(standardType, _) => decodePrimitive(path, standardType) + case Schema.Tuple2(left, right, _) => decodeTuple(path, left, right) + case optionalSchema @ Schema.Optional(_, _) => decodeOptional(path, optionalSchema) + case Schema.Fail(message, _) => fail(path, message) + case Schema.Either(left, right, _) => decodeEither(path, left, right) + case Schema.Fallback(left, right, fullDecode, _) => decodeFallback(path, left, right, fullDecode) + case lzy @ Schema.Lazy(_) => decodeValue(path, lzy.schema) //case Schema.Meta(_, _) => decode(path, Schema[MetaSchema]).map(_.toSchema) case s: Schema.CaseClass0[A] => caseClass0Decoder(path, s) case s: Schema.CaseClass1[_, A] => caseClass1Decoder(path, s) @@ -312,6 +313,30 @@ private[codec] class MessagePackDecoder(bytes: Chunk[Byte]) { } ) + private def decodeFallback[A, B](path: Path, left: Schema[A], right: Schema[B], fullDecode: Boolean): Result[Fallback[A, B]] = + Try(unpacker.unpackArrayHeader()).fold( + err => fail(path, s"Error parsing Fallback structure: ${err.getMessage}"), + size => + if (size == 2) { + decodeString(path :+ "fallback").flatMap { + case "left" => decodeValue(path :+ "fallback:left", left).map(Fallback.Left(_)) + case "right" => decodeValue(path :+ "fallback:right", right).map(Fallback.Right(_)) + case str => fail(path :+ "either", s"Unexpected field name: $str") + } + } else if (size == 3) { + unpacker.skipValue() + if (fullDecode) + decodeValue(path :+ "fallback:left", left).flatMap(l => decodeValue(path :+ "fallback:right", right).map(r => Fallback.Both(l, r))) + else { + val res = decodeValue(path :+ "fallback:left", left).map(Fallback.Left(_)) + unpacker.skipValue() + res + } + } else { + fail(path :+ "fallback", s"Expected 1 or 2 elements but received $size.") + } + ) + private def decodeEnum[Z, A](path: Path, cases: Schema.Case[Z, _]*): Result[Z] = decodeInt(path).flatMap { caseIndex => if (caseIndex > cases.length) { diff --git a/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackEncoder.scala b/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackEncoder.scala index ca267d2fc..e7e6097ae 100644 --- a/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackEncoder.scala +++ b/zio-schema-msg-pack/src/main/scala/zio/schema/codec/MessagePackEncoder.scala @@ -8,7 +8,7 @@ import scala.collection.immutable.ListMap import org.msgpack.core.MessagePack import zio.Chunk -import zio.schema.{ DynamicValue, Schema, StandardType } +import zio.schema.{ DynamicValue, Fallback, Schema, StandardType } private[codec] class MessagePackEncoder { private val packer = MessagePack.newDefaultBufferPacker() @@ -32,6 +32,7 @@ private[codec] class MessagePackEncoder { case (Schema.Tuple2(left, right, _), v @ (_, _)) => encodeTuple(left, right, v) case (optSchema: Schema.Optional[_], v: Option[_]) => encodeOptional(optSchema.asInstanceOf[Schema.Optional[Any]].schema, v.asInstanceOf[Option[Any]]) case (eitherSchema: Schema.Either[_, _], v: scala.util.Either[_, _]) => encodeEither(eitherSchema.asInstanceOf[Schema.Either[Any, Any]].left, eitherSchema.asInstanceOf[Schema.Either[Any, Any]].right, v.asInstanceOf[scala.util.Either[Any, Any]]) + case (fallbackSchema: Schema.Fallback[_, _], v: Fallback[_, _]) => encodeFallback(fallbackSchema.asInstanceOf[Schema.Fallback[Any, Any]].left, fallbackSchema.asInstanceOf[Schema.Fallback[Any, Any]].right, v.asInstanceOf[Fallback[Any, Any]]) case (lzy @ Schema.Lazy(_), v) => encodeValue(lzy.schema, v) // case (Schema.Meta(ast, _), _) => encodeValue(fieldNumber, Schema[MetaSchema], ast) case (Schema.CaseClass0(_, _, _), _) => encodePrimitive(StandardType.UnitType, ()) @@ -140,6 +141,23 @@ private[codec] class MessagePackEncoder { } } + private def encodeFallback[A, B](left: Schema[A], right: Schema[B], fallback: Fallback[A, B]): Unit = + fallback match { + case Fallback.Left(value) => + packer.packArrayHeader(2) + packer.packString("left") + encodeValue(left, value) + case Fallback.Right(value) => + packer.packArrayHeader(2) + packer.packString("right") + encodeValue(right, value) + case Fallback.Both(valueLeft, valueRight) => + packer.packArrayHeader(3) + packer.packString("both") + encodeValue(left, valueLeft) + encodeValue(right, valueRight) + } + private def encodeCaseClass[Z](value: Z, fields: (Schema.Field[Z, _])*): Unit = writeStructure(fields.map { field => (field, field.get(value)) diff --git a/zio-schema-msg-pack/src/test/scala-2/zio/schema/codec/MessagePackCodecSpec.scala b/zio-schema-msg-pack/src/test/scala-2/zio/schema/codec/MessagePackCodecSpec.scala index dd3886b42..fd99401cc 100644 --- a/zio-schema-msg-pack/src/test/scala-2/zio/schema/codec/MessagePackCodecSpec.scala +++ b/zio-schema-msg-pack/src/test/scala-2/zio/schema/codec/MessagePackCodecSpec.scala @@ -25,7 +25,7 @@ object MessagePackCodecSpec extends ZIOSpecDefault { val objectMapper = new ObjectMapper(new MessagePackFactory()) objectMapper.registerModule(DefaultScalaModule) - def spec: Spec[TestEnvironment with Scope, Any] = suite("ThriftCodec Spec")( + def spec: Spec[TestEnvironment with Scope, Any] = suite("MessagePackCodec Spec")( suite("Should correctly encode")( test("integers") { for { @@ -434,6 +434,63 @@ object MessagePackCodecSpec extends ZIOSpecDefault { ed2 <- encodeAndDecodeNS(complexEitherSchema, eitherRight) } yield assert(ed)(equalTo(Chunk(eitherRight2))) && assert(ed2)(equalTo(eitherRight)) }, + test("fallback left full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback left non full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right non full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both non full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback.simplify))) && assert(ed2)(equalTo(fallback.simplify)) + }, + test("fallback with product type") { + val fallbackLeft = zio.schema.Fallback.Left(MyRecord(150)) + for { + ed <- encodeAndDecode(complexFallbackSchema2, fallbackLeft) + ed2 <- encodeAndDecodeNS(complexFallbackSchema2, fallbackLeft) + } yield assert(ed)(equalTo(Chunk(fallbackLeft))) && assert(ed2)(equalTo(fallbackLeft)) + }, + test("fallback with sum type") { + val fallbackRight = zio.schema.Fallback.Right(BooleanValue(true)) + val fallbackRight2 = zio.schema.Fallback.Right(StringValue("hello")) + for { + ed <- encodeAndDecode(complexFallbackSchema, fallbackRight2) + ed2 <- encodeAndDecodeNS(complexFallbackSchema, fallbackRight) + } yield assert(ed)(equalTo(Chunk(fallbackRight2))) && assert(ed2)(equalTo(fallbackRight)) + }, test("optionals") { val value = Some(123) for { @@ -902,6 +959,16 @@ object MessagePackCodecSpec extends ZIOSpecDefault { val complexEitherSchema2: Schema.Either[MyRecord, MyRecord] = Schema.Either(myRecord, myRecord) + val fallbackSchema1: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], true) + + val fallbackSchema2: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], false) + + val complexFallbackSchema: Schema.Fallback[Record, OneOf] = + Schema.Fallback(Record.schemaRecord, schemaOneOf) + + val complexFallbackSchema2: Schema.Fallback[MyRecord, MyRecord] = + Schema.Fallback(myRecord, myRecord) + case class RichProduct(stringOneOf: OneOf, basicString: BasicString, record: Record) lazy val richProductSchema: Schema[RichProduct] = DeriveSchema.gen[RichProduct] diff --git a/zio-schema-protobuf/shared/src/main/scala/zio/schema/codec/ProtobufCodec.scala b/zio-schema-protobuf/shared/src/main/scala/zio/schema/codec/ProtobufCodec.scala index f65d10924..e319d2b8d 100644 --- a/zio-schema-protobuf/shared/src/main/scala/zio/schema/codec/ProtobufCodec.scala +++ b/zio-schema-protobuf/shared/src/main/scala/zio/schema/codec/ProtobufCodec.scala @@ -61,6 +61,7 @@ object ProtobufCodec { case _: Schema.Optional[_] => false case _: Schema.Fail[_] => false case _: Schema.Either[_, _] => false + case _: Schema.Fallback[_, _] => false case lzy @ Schema.Lazy(_) => canBePacked(lzy.schema) case _ => false } @@ -221,6 +222,19 @@ object ProtobufCodec { encodeKey(WireType.LengthDelimited(encodedEither.size), context.fieldNumber) ++ encodedEither } + override protected def processFallback( + context: EncoderContext, + schema: Schema.Fallback[_, _], + value: Fallback[Chunk[Byte], Chunk[Byte]] + ): Chunk[Byte] = { + val encodedEither = value match { + case Fallback.Left(left) => left + case Fallback.Right(right) => right + case Fallback.Both(left, right) => left ++ right + } + encodeKey(WireType.LengthDelimited(encodedEither.size), context.fieldNumber) ++ encodedEither + } + override protected def processOption( context: EncoderContext, schema: Schema.Optional[_], @@ -278,6 +292,13 @@ object ProtobufCodec { case Right(_) => context.copy(fieldNumber = Some(2)) } + override protected def contextForFallback(context: EncoderContext, f: Fallback[Unit, Unit]): EncoderContext = + f match { + case Fallback.Left(_) => context.copy(fieldNumber = Some(1)) + case Fallback.Right(_) => context.copy(fieldNumber = Some(2)) + case Fallback.Both(_, _) => context.copy(fieldNumber = Some(3)) + } + override protected def contextForOption(context: EncoderContext, o: Option[Unit]): EncoderContext = o match { case None => context.copy(fieldNumber = Some(1)) @@ -893,6 +914,39 @@ object ProtobufCodec { ): Any = value + override protected def startCreatingFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _] + ): Fallback[DecoderContext, DecoderContext] = + keyDecoder(context) match { + case (_, fieldNumber) if fieldNumber == 1 => Fallback.Left(context) + case (_, fieldNumber) if fieldNumber == 2 => Fallback.Right(context) + case (_, fieldNumber) if fieldNumber == 3 => Fallback.Both(context, context) + case _ => + throw ExtraFields(fieldNumber.toString, s"Invalid field number ($fieldNumber) for fallback") + } + + override protected def startReadingRightFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _] + ): DecoderContext = + keyDecoder(context) match { + case (wt, 2) => + wt match { + case LengthDelimited(width) => context.limitedTo(state, width) + case _ => context + } + case (_, fieldNumber) => + throw MalformedField(schema, s"Invalid field number $fieldNumber for fallback's right field") + } + + override protected def createFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _], + value: Fallback[Any, Any] + ): Any = + if (schema.fullDecode) value else value.simplify + override protected def startCreatingTuple(context: DecoderContext, schema: Schema.Tuple2[_, _]): DecoderContext = enterFirstTupleElement(context, schema) diff --git a/zio-schema-protobuf/shared/src/test/scala-2/zio/schema/codec/ProtobufCodecSpec.scala b/zio-schema-protobuf/shared/src/test/scala-2/zio/schema/codec/ProtobufCodecSpec.scala index 431a2c3de..c1c346709 100644 --- a/zio-schema-protobuf/shared/src/test/scala-2/zio/schema/codec/ProtobufCodecSpec.scala +++ b/zio-schema-protobuf/shared/src/test/scala-2/zio/schema/codec/ProtobufCodecSpec.scala @@ -496,6 +496,63 @@ object ProtobufCodecSpec extends ZIOSpecDefault { ed2 <- encodeAndDecodeNS(complexEitherSchema, eitherRight) } yield assert(ed)(equalTo(Chunk(eitherRight2))) && assert(ed2)(equalTo(eitherRight)) }, + test("fallback left full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback left non full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right non full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both non full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback.simplify))) && assert(ed2)(equalTo(fallback.simplify)) + }, + test("fallback with product type") { + val fallbackLeft = zio.schema.Fallback.Left(MyRecord(150)) + for { + ed <- encodeAndDecode(complexFallbackSchema2, fallbackLeft) + ed2 <- encodeAndDecodeNS(complexFallbackSchema2, fallbackLeft) + } yield assert(ed)(equalTo(Chunk(fallbackLeft))) && assert(ed2)(equalTo(fallbackLeft)) + }, + test("fallback with sum type") { + val fallbackRight = zio.schema.Fallback.Right(BooleanValue(true)) + val fallbackRight2 = zio.schema.Fallback.Right(StringValue("hello")) + for { + ed <- encodeAndDecode(complexFallbackSchema, fallbackRight2) + ed2 <- encodeAndDecodeNS(complexFallbackSchema, fallbackRight) + } yield assert(ed)(equalTo(Chunk(fallbackRight2))) && assert(ed2)(equalTo(fallbackRight)) + }, test("optionals") { check(Gen.option(Gen.int(Int.MinValue, Int.MaxValue))) { value => for { @@ -999,6 +1056,16 @@ object ProtobufCodecSpec extends ZIOSpecDefault { val complexEitherSchema2: Schema.Either[MyRecord, MyRecord] = Schema.Either(myRecord, myRecord) + val fallbackSchema1: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], true) + + val fallbackSchema2: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], false) + + val complexFallbackSchema: Schema.Fallback[Record, OneOf] = + Schema.Fallback(Record.schemaRecord, schemaOneOf) + + val complexFallbackSchema2: Schema.Fallback[MyRecord, MyRecord] = + Schema.Fallback(myRecord, myRecord) + case class RichProduct(stringOneOf: OneOf, basicString: BasicString, record: Record) lazy val richProductSchema: Schema[RichProduct] = DeriveSchema.gen[RichProduct] diff --git a/zio-schema-thrift/src/main/scala/zio/schema/codec/ThriftCodec.scala b/zio-schema-thrift/src/main/scala/zio/schema/codec/ThriftCodec.scala index f659b20ae..bae352447 100644 --- a/zio-schema-thrift/src/main/scala/zio/schema/codec/ThriftCodec.scala +++ b/zio-schema-thrift/src/main/scala/zio/schema/codec/ThriftCodec.scala @@ -143,6 +143,16 @@ object ThriftCodec { ): Unit = writeFieldEnd() + override protected def startProcessingFallback(context: Context, schema: Schema.Fallback[_, _]): Unit = + writeFieldBegin(context.fieldNumber, TType.STRUCT) + + override protected def processFallback( + context: Context, + schema: Schema.Fallback[_, _], + value: Fallback[Unit, Unit] + ): Unit = + writeFieldEnd() + override def startProcessingOption(context: Context, schema: Schema.Optional[_]): Unit = writeFieldBegin(context.fieldNumber, TType.STRUCT) @@ -190,6 +200,13 @@ object ThriftCodec { case Right(_) => context.copy(fieldNumber = Some(2)) } + override protected def contextForFallback(context: Context, f: Fallback[Unit, Unit]): Context = + f match { + case Fallback.Left(_) => context.copy(fieldNumber = Some(1)) + case Fallback.Right(_) => context.copy(fieldNumber = Some(2)) + case Fallback.Both(_, _) => context.copy(fieldNumber = Some(3)) + } + override protected def contextForOption(context: Context, o: Option[Unit]): Context = o match { case None => context.copy(fieldNumber = Some(1)) @@ -789,6 +806,39 @@ object ThriftCodec { ): Any = value + override protected def startCreatingFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _] + ): Fallback[DecoderContext, DecoderContext] = { + val readField = p.readFieldBegin() + readField.id match { + case 1 => Fallback.Left(context.copy(path = context.path :+ "fallback:left")) + case 2 => Fallback.Right(context.copy(path = context.path :+ "fallback:right")) + case 3 => + Fallback.Both( + context.copy(path = context.path :+ "fallback:left"), + context.copy(path = context.path :+ "fallback:right") + ) + case _ => fail(context, "Failed to decode fallback.").asInstanceOf[Fallback[DecoderContext, DecoderContext]] + } + } + + override protected def startReadingRightFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _] + ): DecoderContext = { + p.readFieldBegin() + context + } + + override protected def createFallback( + context: DecoderContext, + schema: Schema.Fallback[_, _], + value: Fallback[Any, Any] + ): Any = + value + //if (schema.fullDecode) value else value.simplify + override protected def startCreatingTuple(context: DecoderContext, schema: Schema.Tuple2[_, _]): DecoderContext = { p.readFieldBegin() context diff --git a/zio-schema-thrift/src/test/scala-2/zio/schema/codec/ThriftCodecSpec.scala b/zio-schema-thrift/src/test/scala-2/zio/schema/codec/ThriftCodecSpec.scala index 67c4cd9f5..a45fb452b 100644 --- a/zio-schema-thrift/src/test/scala-2/zio/schema/codec/ThriftCodecSpec.scala +++ b/zio-schema-thrift/src/test/scala-2/zio/schema/codec/ThriftCodecSpec.scala @@ -486,6 +486,63 @@ object ThriftCodecSpec extends ZIOSpecDefault { ed2 <- encodeAndDecodeNS(complexEitherSchema, eitherRight) } yield assert(ed)(equalTo(Chunk(eitherRight2))) && assert(ed2)(equalTo(eitherRight)) }, + test("fallback left full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback left non full decode") { + val fallback = zio.schema.Fallback.Left(9) + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback right non full decode") { + val fallback = zio.schema.Fallback.Right("hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema1, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema1, fallback) + } yield assert(ed)(equalTo(Chunk(fallback))) && assert(ed2)(equalTo(fallback)) + }, + test("fallback both non full decode") { + val fallback = zio.schema.Fallback.Both(2, "hello") + for { + ed <- encodeAndDecode(fallbackSchema2, fallback) + ed2 <- encodeAndDecodeNS(fallbackSchema2, fallback) + } yield assert(ed)(equalTo(Chunk(fallback.simplify))) && assert(ed2)(equalTo(fallback.simplify)) + }, + test("fallback with product type") { + val fallbackLeft = zio.schema.Fallback.Left(MyRecord(150)) + for { + ed <- encodeAndDecode(complexFallbackSchema2, fallbackLeft) + ed2 <- encodeAndDecodeNS(complexFallbackSchema2, fallbackLeft) + } yield assert(ed)(equalTo(Chunk(fallbackLeft))) && assert(ed2)(equalTo(fallbackLeft)) + }, + test("fallback with sum type") { + val fallbackRight = zio.schema.Fallback.Right(BooleanValue(true)) + val fallbackRight2 = zio.schema.Fallback.Right(StringValue("hello")) + for { + ed <- encodeAndDecode(complexFallbackSchema, fallbackRight2) + ed2 <- encodeAndDecodeNS(complexFallbackSchema, fallbackRight) + } yield assert(ed)(equalTo(Chunk(fallbackRight2))) && assert(ed2)(equalTo(fallbackRight)) + }, test("optionals") { val value = Some(123) for { @@ -1028,6 +1085,16 @@ object ThriftCodecSpec extends ZIOSpecDefault { val complexEitherSchema2: Schema.Either[MyRecord, MyRecord] = Schema.Either(myRecord, myRecord) + val fallbackSchema1: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], true) + + val fallbackSchema2: Schema.Fallback[Int, String] = Schema.Fallback(Schema[Int], Schema[String], false) + + val complexFallbackSchema: Schema.Fallback[Record, OneOf] = + Schema.Fallback(Record.schemaRecord, schemaOneOf) + + val complexFallbackSchema2: Schema.Fallback[MyRecord, MyRecord] = + Schema.Fallback(myRecord, myRecord) + case class RichProduct(stringOneOf: OneOf, basicString: BasicString, record: Record) lazy val richProductSchema: Schema[RichProduct] = DeriveSchema.gen[RichProduct] diff --git a/zio-schema/shared/src/main/scala/zio/schema/Differ.scala b/zio-schema/shared/src/main/scala/zio/schema/Differ.scala index 437a567d4..c7bc8b27e 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/Differ.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/Differ.scala @@ -262,6 +262,7 @@ object Differ { case Schema.Set(s, _) => set(s) case Schema.Map(k, v, _) => map(k, v) case Schema.Either(leftSchema, rightSchema, _) => either(fromSchema(leftSchema), fromSchema(rightSchema)) + case Schema.Fallback(leftSchema, rightSchema, _, _) => fallback(fromSchema(leftSchema), fromSchema(rightSchema)) case s @ Schema.Lazy(_) => fromSchema(s.schema) case Schema.Transform(schema, g, f, _, _) => fromSchema(schema).transformOrFail(f, g) case Schema.Fail(_, _) => fail @@ -535,6 +536,13 @@ object Differ { case (Right(l), Right(r)) => Patch.EitherDiff(Right(right(l, r))) } + def fallback[A, B](left: Differ[A], right: Differ[B]): Differ[Fallback[A, B]] = + instancePartial[Fallback[A, B]] { + case (Fallback.Left(l), Fallback.Left(r)) => Patch.Fallback(Fallback.Left(left(l, r))) + case (Fallback.Right(l), Fallback.Right(r)) => Patch.Fallback(Fallback.Right(right(l, r))) + case (Fallback.Both(l1, r1), Fallback.Both(l2, r2)) => Patch.Fallback(Fallback.Both(left(l1, l2), right(r1, r2))) + } + def identical[A]: Differ[A] = (_: A, _: A) => Patch.identical def fail[A]: Differ[A] = (_: A, _: A) => Patch.notComparable diff --git a/zio-schema/shared/src/main/scala/zio/schema/DynamicValue.scala b/zio-schema/shared/src/main/scala/zio/schema/DynamicValue.scala index a16993281..f06358f0f 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/DynamicValue.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/DynamicValue.scala @@ -154,6 +154,16 @@ object DynamicValue { case Right(value) => DynamicValue.RightValue(value) } + override protected def processFallback( + schema: Schema.Fallback[_, _], + value: Fallback[DynamicValue, DynamicValue] + ): DynamicValue = + value match { + case Fallback.Left(value) => DynamicValue.LeftValue(value) + case Fallback.Right(value) => DynamicValue.RightValue(value) + case Fallback.Both(left, right) => DynamicValue.BothValue(left, right) + } + override protected def processOption(schema: Schema.Optional[_], value: Option[DynamicValue]): DynamicValue = value match { case Some(value) => DynamicValue.SomeValue(value) @@ -221,6 +231,8 @@ object DynamicValue { final case class RightValue(value: DynamicValue) extends DynamicValue + final case class BothValue(left: DynamicValue, right: DynamicValue) extends DynamicValue + final case class DynamicAst(ast: MetaSchema) extends DynamicValue final case class Error(message: String) extends DynamicValue diff --git a/zio-schema/shared/src/main/scala/zio/schema/Fallback.scala b/zio-schema/shared/src/main/scala/zio/schema/Fallback.scala new file mode 100644 index 000000000..a0210274c --- /dev/null +++ b/zio-schema/shared/src/main/scala/zio/schema/Fallback.scala @@ -0,0 +1,63 @@ +package zio.schema + +/** + * `Fallback` represents an enriched `Either` type that can contain both a left and a right value. The left value represents the default value, that fallbacks to the right value when it is not found. + */ +sealed trait Fallback[+A, +B] { self => + + /** + * Tranform a `Fallback` into an `Either`, using the left value for `Fallback.Both`. + */ + def toEither: Either[A, B] = self match { + case Fallback.Left(l) => Left(l) + case Fallback.Right(r) => Right(r) + case Fallback.Both(l, _) => Left(l) + } + + /** + * Deletes the right part of `Fallback.Both` instances. + */ + def simplify: Fallback[A, B] = self match { + case Fallback.Both(left, _) => Fallback.Left(left) + case other => other + } + + def fold[C](fa: A => C, fb: B => C): C = self match { + case Fallback.Left(left) => fa(left) + case Fallback.Right(right) => fb(right) + case Fallback.Both(left, _) => fa(left) + } + + def map[C](f: A => C): Fallback[C, B] = mapLeft(f) + + def mapLeft[C](f: A => C): Fallback[C, B] = self match { + case Fallback.Left(left) => Fallback.Left(f(left)) + case Fallback.Right(right) => Fallback.Right(right) + case Fallback.Both(left, right) => Fallback.Both(f(left), right) + } + + def mapRight[C](f: B => C): Fallback[A, C] = self match { + case Fallback.Left(left) => Fallback.Left(left) + case Fallback.Right(right) => Fallback.Right(f(right)) + case Fallback.Both(left, right) => Fallback.Both(left, f(right)) + } + + def swap: Fallback[B, A] = self match { + case Fallback.Left(left) => Fallback.Right(left) + case Fallback.Right(right) => Fallback.Left(right) + case Fallback.Both(left, right) => Fallback.Both(right, left) + } +} + +object Fallback { + + def fromEither[A, B](either: Either[A, B]): Fallback[A, B] = either match { + case scala.util.Left(value) => Left(value) + case scala.util.Right(value) => Right(value) + } + + final case class Left[+A, +B](left: A) extends Fallback[A, B] + final case class Right[+A, +B](right: B) extends Fallback[A, B] + final case class Both[+A, +B](left: A, right: B) extends Fallback[A, B] + +} diff --git a/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueBuilder.scala b/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueBuilder.scala index 6c83017c8..9234ffbfd 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueBuilder.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueBuilder.scala @@ -130,6 +130,18 @@ trait MutableSchemaBasedValueBuilder[Target, Context] { /** Create the either value from an inner value */ protected def createEither(context: Context, schema: Schema.Either[_, _], value: Either[Target, Target]): Target + /** The next value to be created is a fallback value with the given Fallback schema. Similarly to optional values, + * this method is responsible for gathering enough information to decide whether the created value will + * be a Left, a Right or a Both. The result value represents this, and for each case allows specifying a context + * that will be used to create the inner value. */ + protected def startCreatingFallback(context: Context, schema: Schema.Fallback[_, _]): Fallback[Context, Context] + + /** Create the fallback value from an inner value */ + protected def createFallback(context: Context, schema: Schema.Fallback[_, _], value: Fallback[Target, Target]): Target + + /** Called for reading right element of a `Fallback.Both` */ + protected def startReadingRightFallback(context: Context, schema: Schema.Fallback[_, _]): Context + /** The next value to be created is a tuple with the given schema. The returned context is used to * construct the first element of the tuple. */ protected def startCreatingTuple(context: Context, schema: Schema.Tuple2[_, _]): Context @@ -638,6 +650,43 @@ trait MutableSchemaBasedValueBuilder[Target, Context] { } } + case s: Schema.Fallback[l, r] => + startCreatingFallback(currentContext, s) match { + case Fallback.Left(newState) => + currentSchema = s.left + pushContext(newState) + push { value => + contextStack = contextStack.tail + finishWith(createFallback(contextStack.head, s, Fallback.Left(value))) + } + case Fallback.Right(newState) => + currentSchema = s.right + pushContext(newState) + push { value => + contextStack = contextStack.tail + finishWith(createFallback(contextStack.head, s, Fallback.Right(value))) + } + case Fallback.Both(leftState, rightState) => + currentSchema = s.left + pushContext(leftState) + push { left => + contextStack = contextStack.tail + currentSchema = s.right + pushContext(startReadingRightFallback(rightState, s)) + push { right => + contextStack = contextStack.tail + finishWith( + createFallback( + contextStack.head, + s, + if (s.fullDecode) Fallback.Both(left, right) + else Fallback.Left(left) + ) + ) + } + } + } + case s: Schema.Tuple2[a, b] => currentSchema = s.left pushContext(startCreatingTuple(currentContext, s)) @@ -1144,6 +1193,25 @@ trait SimpleMutableSchemaBasedValueBuilder[Target] extends MutableSchemaBasedVal protected def createEither(schema: Schema.Either[_, _], value: Either[Target, Target]): Target + override protected def startCreatingFallback(context: Unit, schema: Schema.Fallback[_, _]): Fallback[Unit, Unit] = + startCreatingFallback(schema) + + protected def startCreatingFallback(schema: Schema.Fallback[_, _]): Fallback[Unit, Unit] + + override protected def startReadingRightFallback(context: Unit, schema: Schema.Fallback[_, _]): Unit = + startReadingRightFallback(schema) + + protected def startReadingRightFallback(schema: Schema.Fallback[_, _]): Unit + + override protected def createFallback( + context: Unit, + schema: Schema.Fallback[_, _], + value: zio.schema.Fallback[Target, Target] + ): Target = + createFallback(schema, value) + + protected def createFallback(schema: Schema.Fallback[_, _], value: zio.schema.Fallback[Target, Target]): Target + override protected def startCreatingTuple(context: Unit, schema: Schema.Tuple2[_, _]): Unit = startCreatingTuple(schema) diff --git a/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueProcessor.scala b/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueProcessor.scala index 3c1bb24bf..044e02ad3 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueProcessor.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/MutableSchemaBasedValueProcessor.scala @@ -54,12 +54,22 @@ trait MutableSchemaBasedValueProcessor[Target, Context] { /** Process a set using its already processed elements */ protected def processSet(context: Context, schema: Schema.Set[_], value: Set[Target]): Target - /** Called before processing and either value */ + /** Called before processing an either value */ protected def startProcessingEither(context: Context, schema: Schema.Either[_, _]): Unit = {} /** Process an either value using its already processed left or right value */ protected def processEither(context: Context, schema: Schema.Either[_, _], value: Either[Target, Target]): Target + /** Called before processing a fallback value */ + protected def startProcessingFallback(context: Context, schema: Schema.Fallback[_, _]): Unit = {} + + /** Process a fallback value using its already processed left or right value */ + protected def processFallback( + context: Context, + schema: Schema.Fallback[_, _], + value: zio.schema.Fallback[Target, Target] + ): Target + /** Called before processing an option value */ protected def startProcessingOption(context: Context, schema: Schema.Optional[_]): Unit = {} @@ -94,6 +104,9 @@ trait MutableSchemaBasedValueProcessor[Target, Context] { /** Gets the context for an either's left or right value within the parent context */ protected def contextForEither(context: Context, e: Either[Unit, Unit]): Context + /** Gets the context for a fallback's left or right value within the parent context */ + protected def contextForFallback(context: Context, e: zio.schema.Fallback[Unit, Unit]): Context + /** Gets the context for an option's inner value within the parent context */ protected def contextForOption(context: Context, o: Option[Unit]): Context @@ -834,6 +847,41 @@ trait MutableSchemaBasedValueProcessor[Target, Context] { } } + case s: Schema.Fallback[l, r] => + startProcessingFallback(currentContext, s) + currentValue.asInstanceOf[zio.schema.Fallback[l, r]] match { + case zio.schema.Fallback.Left(value: l) => + currentValue = value + currentSchema = s.left + pushContext(contextForFallback(currentContext, zio.schema.Fallback.Left(()))) + push { dyn => + contextStack = contextStack.tail + finishWith(processFallback(currentContext, s, zio.schema.Fallback.Left(dyn))) + } + case zio.schema.Fallback.Right(value: r) => + currentValue = value + currentSchema = s.right + pushContext(contextForFallback(currentContext, zio.schema.Fallback.Right(()))) + push { dyn => + contextStack = contextStack.tail + finishWith(processFallback(currentContext, s, zio.schema.Fallback.Right(dyn))) + } + case zio.schema.Fallback.Both(left, right) => + currentValue = left + currentSchema = s.left + pushContext(contextForFallback(currentContext, zio.schema.Fallback.Both((), ()))) + push { dynLeft => + contextStack = contextStack.tail + currentValue = right + currentSchema = s.right + pushContext(contextForFallback(currentContext, zio.schema.Fallback.Right(()))) + push { dynRight => + contextStack = contextStack.tail + finishWith(processFallback(currentContext, s, zio.schema.Fallback.Both(dynLeft, dynRight))) + } + } + } + case s: Schema.Tuple2[a, b] => startProcessingTuple(currentContext, s) val (a: a, b: b) = currentValue.asInstanceOf[(a, b)] @@ -1348,6 +1396,8 @@ trait SimpleMutableSchemaBasedValueProcessor[Target] extends MutableSchemaBasedV protected def processEither(schema: Schema.Either[_, _], value: Either[Target, Target]): Target + protected def processFallback(schema: Schema.Fallback[_, _], value: zio.schema.Fallback[Target, Target]): Target + protected def processOption(schema: Schema.Optional[_], value: Option[Target]): Target protected def processTuple(schema: Schema.Tuple2[_, _], left: Target, right: Target): Target @@ -1393,6 +1443,13 @@ trait SimpleMutableSchemaBasedValueProcessor[Target] extends MutableSchemaBasedV ): Target = processEither(schema, value) + override protected def processFallback( + context: Unit, + schema: Schema.Fallback[_, _], + value: zio.schema.Fallback[Target, Target] + ): Target = + processFallback(schema, value) + override protected def processOption(context: Unit, schema: Schema.Optional[_], value: Option[Target]): Target = processOption(schema, value) @@ -1416,6 +1473,9 @@ trait SimpleMutableSchemaBasedValueProcessor[Target] extends MutableSchemaBasedV override protected def contextForEither(context: Unit, e: Either[Unit, Unit]): Unit = () + override protected def contextForFallback(context: Unit, e: zio.schema.Fallback[Unit, Unit]): Unit = + () + override protected def contextForOption(context: Unit, o: Option[Unit]): Unit = () diff --git a/zio-schema/shared/src/main/scala/zio/schema/Patch.scala b/zio-schema/shared/src/main/scala/zio/schema/Patch.scala index 5d5e7e70b..5dcee2e3c 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/Patch.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/Patch.scala @@ -266,6 +266,49 @@ object Patch { } } + final case class Fallback[A, B](diff: zio.schema.Fallback[Patch[A], Patch[B]]) + extends Patch[zio.schema.Fallback[A, B]] { + override def isIdentical: Boolean = diff.fold(_.isIdentical, _.isIdentical) + + override def isComparable: Boolean = diff.fold(_.isComparable, _.isComparable) + + override def patch(input: zio.schema.Fallback[A, B]): Either[String, zio.schema.Fallback[A, B]] = + (input, diff) match { + case (zio.schema.Fallback.Left(_), zio.schema.Fallback.Right(_)) => + Left(s"Cannot apply a right diff to a left value") + case (zio.schema.Fallback.Right(_), zio.schema.Fallback.Left(_)) => + Left(s"Cannot apply a left diff to a right value") + case (zio.schema.Fallback.Left(in), zio.schema.Fallback.Left(diff)) => + diff.patch(in).map(zio.schema.Fallback.Left(_)) + case (zio.schema.Fallback.Right(in), zio.schema.Fallback.Right(diff)) => + diff.patch(in).map(zio.schema.Fallback.Right(_)) + + case (zio.schema.Fallback.Both(in, _), zio.schema.Fallback.Left(diff)) => + diff.patch(in).map(zio.schema.Fallback.Left(_)) + case (zio.schema.Fallback.Both(_, in), zio.schema.Fallback.Right(diff)) => + diff.patch(in).map(zio.schema.Fallback.Right(_)) + case (zio.schema.Fallback.Left(in), zio.schema.Fallback.Both(diff, _)) => + diff.patch(in).map(zio.schema.Fallback.Left(_)) + case (zio.schema.Fallback.Right(in), zio.schema.Fallback.Both(_, diff)) => + diff.patch(in).map(zio.schema.Fallback.Right(_)) + case (zio.schema.Fallback.Both(inLeft, inRight), zio.schema.Fallback.Both(diffLeft, diffRight)) => + (diffLeft.patch(inLeft), diffRight.patch(inRight)) match { + case (Right(a), Right(b)) => Right(zio.schema.Fallback.Both(a, b)) + case (Left(_), Right(b)) => Right(zio.schema.Fallback.Right(b)) + case (Right(a), Left(_)) => Right(zio.schema.Fallback.Left(a)) + case (Left(_), Left(_)) => Left(s"Diff wasn't appliable neither in left or right fallback") + + } + + } + + override def invert: Patch[zio.schema.Fallback[A, B]] = diff match { + case zio.schema.Fallback.Left(value) => Fallback(zio.schema.Fallback.Left(value.invert)) + case zio.schema.Fallback.Right(value) => Fallback(zio.schema.Fallback.Right(value.invert)) + case zio.schema.Fallback.Both(left, right) => Fallback(zio.schema.Fallback.Both(left.invert, right.invert)) + } + } + final case class Transform[A, B](patch: Patch[A], f: A => Either[String, B], g: B => Either[String, A]) extends Patch[B] { override def isIdentical: Boolean = patch.isIdentical diff --git a/zio-schema/shared/src/main/scala/zio/schema/Schema.scala b/zio-schema/shared/src/main/scala/zio/schema/Schema.scala index 3bdab6378..1bebd71e1 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/Schema.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/Schema.scala @@ -244,6 +244,12 @@ object Schema extends SchemaPlatformSpecific with SchemaEquality { case Left(value) => loop(value, left) case Right(value) => loop(value, right) } + case fallback @ Schema.Fallback(left, right, _, _) => + value.asInstanceOf[zio.schema.Fallback[fallback.LeftType, fallback.RightType]] match { + case zio.schema.Fallback.Left(value) => loop(value, left) + case zio.schema.Fallback.Right(value) => loop(value, right) + case zio.schema.Fallback.Both(valueLeft, valueRight) => loop(valueLeft, left) ++ loop(valueRight, right) + } case Dynamic(_) => Chunk.empty case Fail(_, _) => Chunk.empty } @@ -305,6 +311,9 @@ object Schema extends SchemaPlatformSpecific with SchemaEquality { implicit def either[A, B](implicit left: Schema[A], right: Schema[B]): Schema[scala.util.Either[A, B]] = Schema.Either(left, right) + implicit def fallback[A, B](implicit left: Schema[A], right: Schema[B]): Schema[zio.schema.Fallback[A, B]] = + Schema.Fallback(left, right) + implicit def list[A](implicit schemaA: Schema[A]): Schema[List[A]] = Schema.Sequence[List[A], A, String](schemaA, _.toList, Chunk.fromIterable(_), Chunk.empty, "List") @@ -637,6 +646,74 @@ object Schema extends SchemaPlatformSpecific with SchemaEquality { } + /** + * Schema for `zio.schema.Fallback` type. If `fullDecode` is set to `true`, it will decode `Fallback.Both` from `Fallback.Both`. + * If set to `false`, it will decode `Fallback.Left` when possible and `Fallback.Right` as second option from a `Fallback.Both`. + */ + final case class Fallback[A, B]( + left: Schema[A], + right: Schema[B], + fullDecode: Boolean = false, + annotations: Chunk[Any] = Chunk.empty + ) extends Schema[zio.schema.Fallback[A, B]] { + self => + type LeftType = A + type RightType = B + + val leftSingleton = "Left" + val rightSingleton = "Right" + override type Accessors[Lens[_, _, _], Prism[_, _, _], Traversal[_, _]] = + ( + Prism[rightSingleton.type, scala.util.Either[A, B], Right[Nothing, B]], + Prism[leftSingleton.type, scala.util.Either[A, B], Left[A, Nothing]] + ) + + override def annotate(annotation: Any): Schema.Fallback[A, B] = copy(annotations = annotations :+ annotation) + + val rightSchema: Schema[Right[Nothing, B]] = right.transform(b => Right(b), _.value) + val leftSchema: Schema[Left[A, Nothing]] = left.transform(a => Left(a), _.value) + + val toEnum: Enum2[Right[Nothing, B], Left[A, Nothing], scala.util.Either[A, B]] = Enum2( + TypeId.parse("zio.schema.Schema.Either"), + Case( + "Right", + rightSchema, + _.asInstanceOf[Right[Nothing, B]], + _.asInstanceOf[scala.util.Either[A, B]], + (e: scala.util.Either[A, B]) => e.isRight, + Chunk.empty + ), + Case( + "Left", + leftSchema, + _.asInstanceOf[Left[A, Nothing]], + _.asInstanceOf[scala.util.Either[A, B]], + (e: scala.util.Either[A, B]) => e.isLeft, + Chunk.empty + ), + Chunk.empty + ) + + val toEither: Either[A, B] = Either[A, B](left, right, annotations) + + override def defaultValue: scala.util.Either[String, zio.schema.Fallback[A, B]] = + (left.defaultValue, right.defaultValue) match { + case (Right(a), Right(b)) => Right(zio.schema.Fallback.Both(a, b)) + case (Right(a), _) => Right(zio.schema.Fallback.Left(a)) + case (_, Right(b)) => Right(zio.schema.Fallback.Right(b)) + case _ => Left("unable to extract default value for Fallback") + } + + override def makeAccessors( + b: AccessorBuilder + ): ( + b.Prism[rightSingleton.type, scala.util.Either[A, B], Right[Nothing, B]], + b.Prism[leftSingleton.type, scala.util.Either[A, B], Left[A, Nothing]] + ) = + b.makePrism(toEnum, toEnum.case1) -> b.makePrism(toEnum, toEnum.case2) + + } + final case class Lazy[A](private val schema0: () => Schema[A]) extends Schema[A] { override type Accessors[Lens[_, _, _], Prism[_, _, _], Traversal[_, _]] = schema.Accessors[Lens, Prism, Traversal] diff --git a/zio-schema/shared/src/main/scala/zio/schema/SchemaEquality.scala b/zio-schema/shared/src/main/scala/zio/schema/SchemaEquality.scala index f30e43402..00715145c 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/SchemaEquality.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/SchemaEquality.scala @@ -70,6 +70,10 @@ trait SchemaEquality { lEither.annotations == rEither.annotations && lEither.left === rEither.left && lEither.right === rEither.right + case (lEither: Schema.Fallback[_, _], rEither: Schema.Fallback[_, _]) => + lEither.annotations == rEither.annotations && + lEither.left === rEither.left && + lEither.right === rEither.right case (lLazy: Schema.Lazy[_], rLazy: Schema.Lazy[_]) => if (lLazy.schema eq rLazy.schema) true diff --git a/zio-schema/shared/src/main/scala/zio/schema/meta/AstRenderer.scala b/zio-schema/shared/src/main/scala/zio/schema/meta/AstRenderer.scala index 37c09de19..d5ac454bf 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/meta/AstRenderer.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/meta/AstRenderer.scala @@ -36,6 +36,14 @@ private[schema] object AstRenderer { .append("\n") .append(Chunk(Labelled("left", left), Labelled("right", right)).map(renderField(_, INDENT_STEP)).mkString("\n")) .toString + case ExtensibleMetaSchema.Fallback(_, left, right, optional) => + val buffer = new StringBuffer() + buffer.append(s"fallback") + if (optional) buffer.append("?") + buffer + .append("\n") + .append(Chunk(Labelled("left", left), Labelled("right", right)).map(renderField(_, INDENT_STEP)).mkString("\n")) + .toString case ExtensibleMetaSchema.ListNode(items, _, optional) => val buffer = new StringBuffer() buffer.append(s"list") @@ -107,6 +115,18 @@ private[schema] object AstRenderer { .mkString("\n") ) .toString + case ExtensibleMetaSchema.Fallback(_, left, right, optional) => + pad(buffer, indent) + buffer.append(s"${labelled.label}: fallback") + if (optional) buffer.append("?") + buffer + .append("\n") + .append( + Chunk(Labelled("left", left), Labelled("right", right)) + .map(renderField(_, indent + INDENT_STEP)) + .mkString("\n") + ) + .toString case ExtensibleMetaSchema.ListNode(items, _, optional) => val buffer = new StringBuffer() buffer.append(s"${labelled.label}: list") diff --git a/zio-schema/shared/src/main/scala/zio/schema/meta/ExtensibleMetaSchema.scala b/zio-schema/shared/src/main/scala/zio/schema/meta/ExtensibleMetaSchema.scala index f9d45fa98..5e8f9f2aa 100644 --- a/zio-schema/shared/src/main/scala/zio/schema/meta/ExtensibleMetaSchema.scala +++ b/zio-schema/shared/src/main/scala/zio/schema/meta/ExtensibleMetaSchema.scala @@ -240,6 +240,58 @@ object ExtensibleMetaSchema { } + final case class Fallback[BuiltIn <: TypeList]( + override val path: NodePath, + left: ExtensibleMetaSchema[BuiltIn], + right: ExtensibleMetaSchema[BuiltIn], + override val optional: Boolean = false + )(implicit val builtInInstances: SchemaInstances[BuiltIn]) + extends ExtensibleMetaSchema[BuiltIn] + + object Fallback { + implicit def schema[BuiltIn <: TypeList]( + implicit builtInInstances: SchemaInstances[BuiltIn] + ): Schema[Fallback[BuiltIn]] = + Schema.CaseClass4( + TypeId.parse("zio.schema.meta.MetaSchema.Fallback"), + field01 = Schema + .Field( + "path", + Schema[String].repeated, + get0 = _.path, + set0 = (a, value: Chunk[String]) => a.copy(path = NodePath(value)) + ), + field02 = Schema + .Field( + "left", + Schema[ExtensibleMetaSchema[BuiltIn]], + get0 = _.left, + set0 = (a, value: ExtensibleMetaSchema[BuiltIn]) => a.copy(left = value) + ), + field03 = Schema + .Field( + "right", + Schema[ExtensibleMetaSchema[BuiltIn]], + get0 = _.right, + set0 = (a, value: ExtensibleMetaSchema[BuiltIn]) => a.copy(right = value) + ), + field04 = Schema + .Field( + "optional", + Schema[Boolean], + get0 = _.optional, + set0 = (a, value: Boolean) => a.copy(optional = value) + ), + ( + path: Chunk[String], + left: ExtensibleMetaSchema[BuiltIn], + right: ExtensibleMetaSchema[BuiltIn], + optional: Boolean + ) => Fallback(NodePath(path), left, right, optional) + ) + + } + final case class FailNode[BuiltIn <: TypeList]( message: String, override val path: NodePath, @@ -507,6 +559,12 @@ object ExtensibleMetaSchema { subtree(NodePath.root / "left", Lineage.empty, left), subtree(NodePath.root / "right", Lineage.empty, right) ) + case Schema.Fallback(left, right, _, _) => + Fallback( + NodePath.root, + subtree(NodePath.root / "left", Lineage.empty, left), + subtree(NodePath.root / "right", Lineage.empty, right) + ) case Schema.Tuple2(left, right, _) => Tuple( NodePath.root, @@ -568,6 +626,13 @@ object ExtensibleMetaSchema { subtree(path / "right", lineage, right, optional = false), optional ) + case Schema.Fallback(left, right, _, _) => + Either( + path, + subtree(path / "left", lineage, left, optional = false), + subtree(path / "right", lineage, right, optional = false), + optional + ) case Schema.Tuple2(left, right, _) => Tuple( path, @@ -667,6 +732,11 @@ object ExtensibleMetaSchema { materialize(left, refs), materialize(right, refs) ) + case ExtensibleMetaSchema.Fallback(_, left, right, _) => + Schema.Fallback( + materialize(left, refs), + materialize(right, refs) + ) case ExtensibleMetaSchema.ListNode(itemAst, _, _) => Schema.chunk(materialize(itemAst, refs)) case ExtensibleMetaSchema.Dictionary(keyAst, valueAst, _, _) =>