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

Cannot read aliased instance in union

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 1.7.6
    • 1.8.0
    • java

    Description

      I created an Avro file using unqualified record names (no namespace) and I want to read them back into another JVM with qualified names. In the second JVM, I supply a schema with the qualified names and an alias to accept the unqualified name. This works as expected if the record is not part of a union, but it fails with an exception if it is part of a union.

      Here is an example that reproduces the bug. In a Scala REPL with avro-1.7.6.jar on the classpath,

      import org.apache.avro.file.DataFileWriter
      import org.apache.avro.generic.GenericData
      import org.apache.avro.generic.GenericDatumWriter
      import org.apache.avro.Schema
      
      val parser = new Schema.Parser
      val schema = parser.parse("""{"type": "record", "name": "Unqualified1", "fields": [{"name": "x", "type": ["null", {"type": "record", "name": "Unqualified2", "fields": [{"name": "y", "type": "string"}]}]}]}}""")
      val unqualified2schema = schema.getField("x").schema.getTypes.get(1)
      
      val unqualified2instance = new GenericData.Record(unqualified2schema)
      unqualified2instance.put("y", "hello")
      val unqualified1instance = new GenericData.Record(schema)
      unqualified1instance.put("x", unqualified2instance)
      
      val datumWriter = new GenericDatumWriter[GenericData.Record](schema)
      val dataFileWriter = new DataFileWriter[GenericData.Record](datumWriter)
      dataFileWriter.create(schema, new java.io.File("tmp.avro"))
      dataFileWriter.append(unqualified1instance)
      dataFileWriter.close()
      

      creates a file that looks like this:

      hexdump -C tmp.avro
      00000000  4f 62 6a 01 02 16 61 76  72 6f 2e 73 63 68 65 6d  |Obj...avro.schem|
      00000010  61 be 02 7b 22 74 79 70  65 22 3a 22 72 65 63 6f  |a..{"type":"reco|
      00000020  72 64 22 2c 22 6e 61 6d  65 22 3a 22 55 6e 71 75  |rd","name":"Unqu|
      00000030  61 6c 69 66 69 65 64 31  22 2c 22 66 69 65 6c 64  |alified1","field|
      00000040  73 22 3a 5b 7b 22 6e 61  6d 65 22 3a 22 78 22 2c  |s":[{"name":"x",|
      00000050  22 74 79 70 65 22 3a 5b  22 6e 75 6c 6c 22 2c 7b  |"type":["null",{|
      00000060  22 74 79 70 65 22 3a 22  72 65 63 6f 72 64 22 2c  |"type":"record",|
      00000070  22 6e 61 6d 65 22 3a 22  55 6e 71 75 61 6c 69 66  |"name":"Unqualif|
      00000080  69 65 64 32 22 2c 22 66  69 65 6c 64 73 22 3a 5b  |ied2","fields":[|
      00000090  7b 22 6e 61 6d 65 22 3a  22 79 22 2c 22 74 79 70  |{"name":"y","typ|
      000000a0  65 22 3a 22 73 74 72 69  6e 67 22 7d 5d 7d 5d 7d  |e":"string"}]}]}|
      000000b0  5d 7d 00 3d 57 38 9b 8c  5a 9a 86 3d b8 18 60 63  |]}.=W8..Z..=..`c|
      000000c0  5c bb c5 02 0e 02 0a 68  65 6c 6c 6f 3d 57 38 9b  |\......hello=W8.|
      000000d0  8c 5a 9a 86 3d b8 18 60  63 5c bb c5              |.Z..=..`c\..|
      000000dc
      

      Now in a new JVM, I try to load it like this:

      import org.apache.avro.file.DataFileReader
      import org.apache.avro.generic.GenericData
      import org.apache.avro.generic.GenericDatumReader
      import org.apache.avro.Schema
      
      val parser = new Schema.Parser
      val schema = parser.parse("""{"type": "record", "namespace": "com.mycompany", "name": "Qualified1", "aliases": ["Unqualified1"], "fields": [{"name": "x", "type": ["null", {"type": "record", "namespace": "com.mycompany", "name": "Qualified2", "aliases": ["Unqualified2"], "fields": [{"name": "y", "type": "string"}]}]}]}}""")
      
      val datumReader = new GenericDatumReader[GenericData.Record](schema)
      val dataFileReader = new DataFileReader[GenericData.Record](new java.io.File("tmp.avro"), datumReader)
      println(dataFileReader.hasNext)   // true
      println(dataFileReader.next())
      

      and get the following exception:

      org.apache.avro.AvroTypeException: Found Unqualified2, expecting union
              at org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:231)
              at org.apache.avro.io.parsing.Parser.advance(Parser.java:88)
              at org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:206)
              at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:155)
              at org.apache.avro.generic.GenericDatumReader.readField(GenericDatumReader.java:193)
              at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:183)
              at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:151)
              at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:142)
              at org.apache.avro.file.DataFileStream.next(DataFileStream.java:233)
              at org.apache.avro.file.DataFileStream.next(DataFileStream.java:220)
              at .<init>(<console>:17)
              at .<clinit>(<console>)
              at .<init>(<console>:7)
              at .<clinit>(<console>)
              at $print(<console>)
              at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
              at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
              at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
              at java.lang.reflect.Method.invoke(Method.java:606)
              at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:734)
              at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:983)
              at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:573)
              at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:604)
              at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:568)
              at scala.tools.nsc.interpreter.ILoop.reallyInterpret$1(ILoop.scala:756)
              at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:801)
              at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:713)
              at scala.tools.nsc.interpreter.ILoop.processLine$1(ILoop.scala:577)
              at scala.tools.nsc.interpreter.ILoop.innerLoop$1(ILoop.scala:584)
              at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:587)
              at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:878)
              at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:833)
              at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:833)
              at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
              at scala.tools.nsc.interpreter.ILoop.process(ILoop.scala:833)
              at scala.tools.nsc.MainGenericRunner.runTarget$1(MainGenericRunner.scala:83)
              at scala.tools.nsc.MainGenericRunner.process(MainGenericRunner.scala:96)
              at scala.tools.nsc.MainGenericRunner$.main(MainGenericRunner.scala:105)
              at scala.tools.nsc.MainGenericRunner.main(MainGenericRunner.scala)
      

      If I do exactly the same thing without a union type, I can read back the original record. Even though this example uses GenericData for simplicity, I first observed the bug using SpecificData.

      (Motivation: I created unqualified record names in one process and then wanted to read it back in another, where I had auto-generated specific classes. I couldn't pollute the parent namespace with the auto-generated classes, and that's why I qualified their names. The union is because sometimes the inner record is legitimately null.)

      Attachments

        Activity

          People

            Unassigned Unassigned
            jpivarski Jim Pivarski
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: