You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Uwe Eisele (Jira)" <ji...@apache.org> on 2021/11/30 21:48:00 UTC

[jira] [Comment Edited] (AVRO-3235) Avro Schema Evolution with Enum – Deserialization Crashes

    [ https://issues.apache.org/jira/browse/AVRO-3235?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17451369#comment-17451369 ] 

Uwe Eisele edited comment on AVRO-3235 at 11/30/21, 9:47 PM:
-------------------------------------------------------------

Hello,

I think I have encountered the same issue with Avro 1.11. However, I have deliberately defined explicitly different namespaces for the enum.

Schema V1:
{code:json}
{"type":"enum","name":"PersonType","namespace":"person.v1","symbols":["UNDEFINED","CUSTOMER","EMPLOYEE"],"default":"UNDEFINED"}
{code}
Schema V2:
{code:json}
{"type":"enum","name":"PersonType","namespace":"person.v2","symbols":["UNDEFINED","CUSTOMER","EMPLOYEE"],"default":"UNDEFINED"}
{code}
The checkReaderWriterCompatibility method indicates that the schemas are compatible.
{code:java}
SchemaCompatibility.SchemaCompatibilityType compatibilityResult =
                SchemaCompatibility.checkReaderWriterCompatibility(SCHEMA_V2, SCHEMA_V2).getResult().getCompatibility();
{code}
However, a deserialization of an enum with V2 which was serialized with V1 fails with the same exception you described.
{code:java}
org.apache.avro.AvroTypeException: Found person.v1.PersonType, expecting person.v2.PersonType
{code}
I would understand that deserialization fails because the Avro specification ([https://avro.apache.org/docs/current/spec.html#names]) states that: "Equality of names is defined on the fullname."
However, I would expect that then the compatibility check would call the schemas incompatible.

The fact that in your example "Found test.simple.v1.Status" is output, although no namespace is specified in schema, is I think due to the fact that the namespace of the parent element is used (the namespace is taken from the most tightly enclosing schema or protocol).

That the namespace is considered during deserialization is apparently not consistent. If a record is used instead of an enum, both the compatibility check would mark the schemas as successful and a corresponding deserialization would work.
However, this contradicts the specification, since the FullName would have to be used. There are already other bug reports about this. AVRO-2793 (https://issues.apache.org/jira/browse/AVRO-2793) describes why the compatibility check does not include the namespace. It refers to a merge request ([https://github.com/apache/avro/pull/526/files]) in which FullName was changed to Name for the check in order to achieve the same behavior as in older versions, although this deviates from the specification.

I can understand this decision, however, I would then expect this behavior for Enums as well. I think there should be no difference between different types here.

For this reason, if there is no other reason not to do so, I would suggest changing the check for enums from FullName to Name during deserialization as well.

Current implementation: ([https://github.com/apache/avro/blob/de50c244c00420825d4bd7d04c0c2d353e439367/lang/java/avro/src/main/java/org/apache/avro/Resolver.java#L391])
{code:java}
public static Action resolve(Schema w, Schema r, GenericData d) {
      if (w.getFullName() != null && !w.getFullName().equals(r.getFullName()))
        return new ErrorAction(w, r, d, ErrorType.NAMES_DONT_MATCH);
{code}
Suggested change, which would allow deserialization with different namespace, but same name:
{code:java}
public static Action resolve(Schema w, Schema r, GenericData d) {
      if (w.getName() != null && !w.getName().equals(r.getName()))
        return new ErrorAction(w, r, d, ErrorType.NAMES_DONT_MATCH);
{code}
It looks there is also no test schema for an enum with a namespace ([https://github.com/apache/avro/blob/de50c244c00420825d4bd7d04c0c2d353e439367/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java#L47]).

What do you think?

Regards,
Uwe


was (Author: ueisele):
Hello,

I think I have encountered the same issue. However, I have deliberately defined explicitly different namespaces for the enum.

Schema V1:
{code:json}
{"type":"enum","name":"PersonType","namespace":"person.v1","symbols":["UNDEFINED","CUSTOMER","EMPLOYEE"],"default":"UNDEFINED"}
{code}
Schema V2:
{code:json}
{"type":"enum","name":"PersonType","namespace":"person.v2","symbols":["UNDEFINED","CUSTOMER","EMPLOYEE"],"default":"UNDEFINED"}
{code}
The checkReaderWriterCompatibility method indicates that the schemas are compatible.
{code:java}
SchemaCompatibility.SchemaCompatibilityType compatibilityResult =
                SchemaCompatibility.checkReaderWriterCompatibility(SCHEMA_V2, SCHEMA_V2).getResult().getCompatibility();
{code}
However, a deserialization of an enum with V2 which was serialized with V1 fails with the same exception you described.
{code:java}
org.apache.avro.AvroTypeException: Found person.v1.PersonType, expecting person.v2.PersonType
{code}
I would understand that deserialization fails because the Avro specification ([https://avro.apache.org/docs/current/spec.html#names]) states that: "Equality of names is defined on the fullname."
However, I would expect that then the compatibility check would call the schemas incompatible.

The fact that in your example "Found test.simple.v1.Status" is output, although no namespace is specified in schema, is I think due to the fact that the namespace of the parent element is used (the namespace is taken from the most tightly enclosing schema or protocol).

That the namespace is considered during deserialization is apparently not consistent. If a record is used instead of an enum, both the compatibility check would mark the schemas as successful and a corresponding deserialization would work.
However, this contradicts the specification, since the FullName would have to be used. There are already other bug reports about this. AVRO-2793 (https://issues.apache.org/jira/browse/AVRO-2793) describes why the compatibility check does not include the namespace. It refers to a merge request ([https://github.com/apache/avro/pull/526/files]) in which FullName was changed to Name for the check in order to achieve the same behavior as in older versions, although this deviates from the specification.

I can understand this decision, however, I would then expect this behavior for Enums as well. I think there should be no difference between different types here.

For this reason, if there is no other reason not to do so, I would suggest changing the check for enums from FullName to Name during deserialization as well.

Current implementation: ([https://github.com/apache/avro/blob/de50c244c00420825d4bd7d04c0c2d353e439367/lang/java/avro/src/main/java/org/apache/avro/Resolver.java#L391])
{code:java}
public static Action resolve(Schema w, Schema r, GenericData d) {
      if (w.getFullName() != null && !w.getFullName().equals(r.getFullName()))
        return new ErrorAction(w, r, d, ErrorType.NAMES_DONT_MATCH);
{code}
Suggested change, which would allow deserialization with different namespace, but same name:
{code:java}
public static Action resolve(Schema w, Schema r, GenericData d) {
      if (w.getName() != null && !w.getName().equals(r.getName()))
        return new ErrorAction(w, r, d, ErrorType.NAMES_DONT_MATCH);
{code}
It looks there is also no test schema for an enum with a namespace (https://github.com/apache/avro/blob/de50c244c00420825d4bd7d04c0c2d353e439367/lang/java/avro/src/test/java/org/apache/avro/TestSchemas.java#L47).

What do you think?

Regards,
Uwe

> Avro Schema Evolution with Enum – Deserialization Crashes
> ---------------------------------------------------------
>
>                 Key: AVRO-3235
>                 URL: https://issues.apache.org/jira/browse/AVRO-3235
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.10.2
>            Reporter: Bertram Beyer
>            Priority: Major
>
> Originally posted on Stack Overflow in June 2020
> https://stackoverflow.com/questions/62596990/avro-schema-evolution-with-enum-deserialization-crashes/
>  
> I defined two versions of a record in two separate AVCS schema files. I used the namespace to distinguish versions *SimpleV1.avsc*
>   
> {code:json}
> {
>   "type" : "record",
>   "name" : "Simple",
>   "namespace" : "test.simple.v1",
>   "fields" : [ 
>       {
>         "name" : "name",
>         "type" : "string"
>       }, 
>       {
>         "name" : "status",
>         "type" : {
>           "type" : "enum",
>           "name" : "Status",
>           "symbols" : [ "ON", "OFF" ]
>         },
>         "default" : "ON"
>       }
>    ]
> }
> {code}
>  
> *Example JSON*
>   
> {code:java}
> {"name":"A","status":"ON"}
> {code}
> Version 2 just has an additional description field with default value.
> *SimpleV2.avsc*
>   
> {code:java}
> {
>   "type" : "record",
>   "name" : "Simple",
>   "namespace" : "test.simple.v2",
>   "fields" : [ 
>       {
>         "name" : "name",
>         "type" : "string"
>       }, 
>       {
>         "name" : "description",
>         "type" : "string",
>         "default" : ""
>       }, 
>       {
>         "name" : "status",
>         "type" : {
>           "type" : "enum",
>           "name" : "Status",
>           "symbols" : [ "ON", "OFF" ]
>         },
>         "default" : "ON"
>       }
>    ]
> }
> {code}
> *Example JSON*
>   
> {code:java}
> {"name":"B","description":"b","status":"ON"}
> {code}
> Both schemas were serialized to Java classes. In my example I was going to test backward compatibility. A record written by V1 shall be read by a reader using V2. I wanted to see that default values are inserted. This is working as long as I do not use enums.
>   
> {code:java}
> public class EnumEvolutionExample {
>     public static void main(String[] args) throws IOException {
>         Schema schemaV1 = new org.apache.avro.Schema.Parser().parse(new File("./src/main/resources/SimpleV1.avsc"));
>         //works as well
>         //Schema schemaV1 = test.simple.v1.Simple.getClassSchema();
>         Schema schemaV2 = new org.apache.avro.Schema.Parser().parse(new File("./src/main/resources/SimpleV2.avsc"));
>         test.simple.v1.Simple simpleV1 = test.simple.v1.Simple.newBuilder()
>                 .setName("A")
>                 .setStatus(test.simple.v1.Status.ON)
>                 .build();
>         
>         
>         SchemaPairCompatibility schemaCompatibility = SchemaCompatibility.checkReaderWriterCompatibility(
>                 schemaV2,
>                 schemaV1);
>         //Checks that writing v1 and reading v2 schemas is compatible
>         Assert.assertEquals(SchemaCompatibilityType.COMPATIBLE, schemaCompatibility.getType());
>         
>         byte[] binaryV1 = serealizeBinary(simpleV1);
>         
>         //Crashes with: AvroTypeException: Found test.simple.v1.Status, expecting test.simple.v2.Status
>         test.simple.v2.Simple v2 = deSerealizeBinary(binaryV1, new test.simple.v2.Simple(), schemaV1);
>         
>     }
>     
>     public static byte[] serealizeBinary(SpecificRecord record) {
>         DatumWriter<SpecificRecord> writer = new SpecificDatumWriter<>(record.getSchema());
>         byte[] data = new byte[0];
>         ByteArrayOutputStream stream = new ByteArrayOutputStream();
>         Encoder binaryEncoder = EncoderFactory.get()
>             .binaryEncoder(stream, null);
>         try {
>             writer.write(record, binaryEncoder);
>             binaryEncoder.flush();
>             data = stream.toByteArray();
>         } catch (IOException e) {
>             System.out.println("Serialization error " + e.getMessage());
>         }
>         return data;
>     }
>     
>     public static <T extends SpecificRecord> T deSerealizeBinary(byte[] data, T reuse, Schema writer) {
>         Decoder decoder = DecoderFactory.get().binaryDecoder(data, null);
>         DatumReader<T> datumReader = new SpecificDatumReader<>(writer, reuse.getSchema());
>         try {
>             T datum = datumReader.read(null, decoder);
>             return datum;
>         } catch (IOException e) {
>             System.out.println("Deserialization error" + e.getMessage());
>         }
>         return null;
>     }
> }
> {code}
> The checkReaderWriterCompatibility method confirms that schemas are compatible. But when I deserialize I’m getting the following exception
>  
> {code:java}
> Exception in thread "main" org.apache.avro.AvroTypeException: Found test.simple.v1.Status, expecting test.simple.v2.Status
>     at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:309)
>     at org.apache.avro.io.parsing.Parser.advance(Parser.java:86)
>     at org.apache.avro.io.ResolvingDecoder.readEnum(ResolvingDecoder.java:260)
>     at org.apache.avro.generic.GenericDatumReader.readEnum(GenericDatumReader.java:267)
>     at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:181)
>     at org.apache.avro.specific.SpecificDatumReader.readField(SpecificDatumReader.java:136)
>     at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:247)
>     at org.apache.avro.specific.SpecificDatumReader.readRecord(SpecificDatumReader.java:123)
>     at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:179)
>     at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:160)
>     at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:153)
>     at test.EnumEvolutionExample.deSerealizeBinary(EnumEvolutionExample.java:70)
>     at test.EnumEvolutionExample.main(EnumEvolutionExample.java:45)
> {code}
>  
> I don’t understand why Avro thinks it got a v1.Status. Namespaces are not part of the encoding.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)