Uploaded image for project: 'Apache Avro'
  1. Apache Avro
  2. AVRO-3408

Schema evolution with logical types

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 1.11.0
    • 1.13.0
    • java

    Description

      Hello!

      First of all, thank you for this project. I love Avro encoding from both technology and code culture points of view. 

      I know you recommend migrating schema by adding a new field and removing the old one in the future, but please-please-please consider my case as well. 

      In my company, we have some DTOs, and it's about 200+ fields in total that we encode with Avro and send over the network. About a third of them have type `java.math.BigDecimal`. At some point, we discovered we send them with a schema like

      {
        "name":"performancePrice",
        "type":{
          "type":"string",
          "java-class":"java.math.BigDecimal"
        }
      }
      

      That's a kind of disaster for us cos we have pretty much a high load with ~2 million RPS.
      So we start to think about migrating to something lighter than strings (no blame for choosing it as a default, I know BigDecimal has a lot of pitfalls, and string is the easiest way for encoding/decoding).
      It was fine to make a standard precision for all such fields, so we found `Conversions.DecimalConversion` and decided at the end of the day we were going to use this logical type with a recommended schema like

          @Override
          public Schema getRecommendedSchema() {
              Schema schema = Schema.create(Schema.Type.BYTES);
              LogicalTypes.Decimal decimalType =
                      LogicalTypes.decimal(MathContext.DECIMAL32.getPrecision(), DecimalUtils.MONEY_ROUNDING_SCALE);
              decimalType.addToSchema(schema);
              return schema;
          }
      

      (we use `org.apache.avro.reflect.ReflectData`)

      It all looks good and promising, but the question is how to migrate to such schema?
      As I said, we have a lot of such fields, and migrating all of them with duplication fields with future removal might be painful and would cost us a considerable overhead.

      I made some tests and found out if two applications register the same `BigDecimalConversion` but for one application the `getRecommendedSchema()` is like the method above and for another application the `getRecommendedSchema()` is

          @Override
          public Schema getRecommendedSchema() {
              Schema schema = Schema.create(Schema.Type.STRING);
              schema.addProp(SpecificData.CLASS_PROP, BigDecimal.class.getName());
              return schema;
          }
      

      so they can easily read each other messages using SERVER schema.

      So, I made two applications and wired them up with `ProtocolRepository`, `ReflectResponder` and all that stuff, I found out it doesn't work. Because `org.apache.avro.io.ResolvingDecoder` totally ignores logical types for some reason.
      So as a result, one application specifically told "I encode this field as a byte array which supposed to be a logical type 'decimal' with precision N", but another application just tries to convert those bytes to a string and make a BigDecimal based on the result string. As a result, we got

      java.lang.NumberFormatException: Character ' is neither a decimal digit number, decimal point, nor "e" notation exponential mark.
      

      In my humble opinion, `org.apache.avro.io.ResolvingDecoder` should respect logical types in SERVER (ACTUAL) schema and use a corresponding conversion instance for reading values. In my example, I'd say it might be

      ResolvingDecoder#readString() -> read the actual logical type -> find BigDecimalConversion instance -> conversion.fromBytes(readValueWithActualSchema()) -> conversion.toCharSequence(readValueWithConversion)
      

      I'd love to read your opinion on all of that.
      Thank you in advance for your time, and sorry for the long issue description.

      Attachments

        1. TestBigDecimalConversion.java
          6 kB
          Christophe Le Saec

        Issue Links

          Activity

            People

              izemlyanskiy Ivan Zemlyanskiy
              izemlyanskiy Ivan Zemlyanskiy
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 6h 20m
                  6h 20m