shiyancao opened a new issue #4281: URL: https://github.com/apache/iceberg/issues/4281
### Issue Hello, Is it possible to support Avro Schema field with default value to Iceberg optional column for schema conversion? In Avro, field with default value is treated as optional. Adding or removing optional field is supported in [full transitive](https://docs.confluent.io/platform/current/schema-registry/avro.html#summary) Avro schema revolution mode. However, when we convert it to Iceberg Schema, it cannot evolve because the column is "required". For full transitive schema evolution, only adding/removing optional column is allowed. I'm aware that we can convert to Iceberg optional column by Union the field with "null" type, but should we also consider closing the discrepancy between Avro and Iceberg Schema by supporting Iceberg default value? --- ### Example: Assuming we have a table with avro schema v1 as: ``` scala> val currentAvroStr = """{ | "type": "record", | "name": "dummy", | "namespace": "example.com", | "fields": [ | { | "name": "f1", | "type": "string", | "default": "a" | } | ] | }""" scala> val currentAvro = new org.apache.iceberg.shaded.org.apache.avro.Schema.Parser().parse(currentAvroStr) currentAvro: org.apache.iceberg.shaded.org.apache.avro.Schema = {"type":"record","name":"dummy","namespace":"example.com","fields":[{"name":"f1","type":"string","default":"a"}]} ``` If we evolve the avro schema to v2 by adding an array field with default value [] ``` scala> val newAvroStr = """{ | "type": "record", | "name": "dummy", | "namespace": "example.com", | "fields": [ | { | "name": "f1", | "type": "string", | "default": "a" | }, | { | "name": "f2", | "type": { | "type": "array", | "items": { | "type": "record", | "name": "arr1", | "fields": [ | { | "name": "f21", | "type": "string", | "doc": "", | "default": "b" | } | ] | } | }, | "doc": "", | "default": [] | } | ] | }""" scala> val newAvro = new org.apache.iceberg.shaded.org.apache.avro.Schema.Parser().parse(newAvroStr) newAvro: org.apache.iceberg.shaded.org.apache.avro.Schema = {"type":"record","name":"dummy","namespace":"example.com","fields":[{"name":"f1","type":"string","default":"a"},{"name":"f2","type":{"type":"array","items":{"type":"record","name":"arr1","fields":[{"name":"f21","type":"string","doc":"","default":"b"}]}},"doc":"","default":[]}]} ``` The evolve is allowed in Avro full transitive mode (no exception is thrown) ``` scala> val validator = new org.apache.iceberg.shaded.org.apache.avro.SchemaValidatorBuilder().mutualReadStrategy().validateAll() validator: org.apache.iceberg.shaded.org.apache.avro.SchemaValidator = org.apache.iceberg.shaded.org.apache.avro.ValidateAll@508228b5 scala> validator.validate(newAvro, List(currentAvro).asJava) ``` #### However, after converting to Iceberg schema, f2 column is now "required" which prevents the schema evolution. ``` scala> val icebergCurrent = AvroSchemaUtil.toIceberg(currentAvro) icebergCurrent: org.apache.iceberg.Schema = table { 0: f1: required string } scala> val icebergNew = AvroSchemaUtil.toIceberg(newAvro) icebergNew: org.apache.iceberg.Schema = table { 0: f1: required string 1: f2: required list<struct<2: f21: required string ()>> () } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
