schema-registry icon indicating copy to clipboard operation
schema-registry copied to clipboard

Connect and Avro Asymmetry with default null fields

Open slominskir opened this issue 2 years ago • 1 comments

It doesn't appear there is currently a way to ensure symmetry when an AVRO Schema is converted to a Connect Schema and then back to AVRO again using the AvroData class when a field is optional, but has no default value. In other words "default": null is not the same as not specifying a default at all, but conversion to Connect and back seems to force a null default. Here is a JUnit test case showing the problem:

    @Test
    public void testNoDefaultValueOnOptionalField() {
        org.apache.avro.Schema expectedAvroSchema = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"TestRecord\",\"namespace\":\"org.test\",\"doc\":\"Test record\",\"fields\":[{\"name\":\"field1\",\"type\":[\"null\",\"int\"]}]}");

        AvroData adTo = new AvroData(new AvroDataConfig.Builder()
                .with(AvroDataConfig.CONNECT_META_DATA_CONFIG, true)
                .build());

        AvroData adFrom = new AvroData(new AvroDataConfig.Builder()
                .with(AvroDataConfig.CONNECT_META_DATA_CONFIG, false)
                .build());

        org.apache.kafka.connect.data.Schema connectSchema = adTo.toConnectSchema(expectedAvroSchema);

        System.out.println("field1 default value: " + connectSchema.field("field1").schema().defaultValue());

        org.apache.avro.Schema actualAvroSchema = adFrom.fromConnectSchema(connectSchema);

        assertEquals(expectedAvroSchema.toString(), actualAvroSchema.toString());
    }

The output comparison error when run (notice default value is added):

Expected :{"type":"record","name":"TestRecord","namespace":"org.test","doc":"Test record","fields":[{"name":"field1","type":["null","int"]}]}
Actual   :{"type":"record","name":"TestRecord","namespace":"org.test","doc":"Test record","fields":[{"name":"field1","type":["null","int"],"default":null}]}

In the test case a separate AvroData instance is used for the to Connect vs the from Connect because the former includes metadata hints, and the later does not. Without metadata hints the doc field is dropped.

This asymmetry is likely because the upstream Kafka Connect Schema API isOptional() and defaultValue() methods appear to be insufficient to determine if a default value is even present (does defaultValue() == null mean no default value or that the default value is null?). Perhaps a isDefaultValue() is needed. In the meantime an AVRO Converter metadata flag could be used. A related KIP suggests a Converter specific flag solution to a similar problem (ambiguity with what a default: null, but in the example given there the field value was null and the confusion is whether a nullable field with null should trigger a default value).

Related:

  • https://github.com/confluentinc/schema-registry/issues/455
  • https://github.com/confluentinc/schema-registry/issues/1693
  • https://federico.is/posts/2020/07/30/avro-unions-and-default-values/

slominskir avatar Apr 25 '22 17:04 slominskir

Note: The workaround in a Connect Transform is to disable attempts to register a new schema and instead use the latest schema version:

    "value.converter.auto.register.schemas": false,
    "value.converter.use.latest.version": true

Without this workaround you'll end up with two versions of the schema: one with an optional field with default: null and one where the optional field does not have a default at all.

slominskir avatar Apr 25 '22 18:04 slominskir