Avro
  1. Avro
  2. AVRO-1467

Schema resolution does not check record names

    Details

    • Type: Bug Bug
    • Status: Open
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: 1.7.6
    • Fix Version/s: 1.8.0
    • Component/s: java
    • Labels:
      None
    • Tags:
      schema-resolution, record

      Description

      According to http://avro.apache.org/docs/1.7.6/spec.html#Schema+Resolution , writer and reader schemae should be considered compatible if they (1) have the same name and (2) the reader requests a subset of the writer's fields with compatible types. In the Java version, I find that the structure of the fields is checked but the name is not checked. (It's too permissive; acts like a structural type check, rather than structural and nominal.)

      Here's a demonstration (in the Scala REPL to allow for experimentation; launch with "scala -cp avro-tools-1.7.6.jar" to get all the classes). The following writes a small, valid Avro data file:

      import org.apache.avro.file.DataFileReader
      import org.apache.avro.file.DataFileWriter
      import org.apache.avro.generic.GenericData
      import org.apache.avro.generic.GenericDatumReader
      import org.apache.avro.generic.GenericDatumWriter
      import org.apache.avro.generic.GenericRecord
      import org.apache.avro.io.DatumReader
      import org.apache.avro.io.DatumWriter
      import org.apache.avro.Schema
      
      val parser = new Schema.Parser
      // The name is different but the fields are the same.
      val writerSchema = parser.parse("""{"type": "record", "name": "Writer", "fields": [{"name": "one", "type": "int"}, {"name": "two", "type": "string"}]}""")
      val readerSchema = parser.parse("""{"type": "record", "name": "Reader", "fields": [{"name": "one", "type": "int"}, {"name": "two", "type": "string"}]}""")
      
      def makeRecord(one: Int, two: String): GenericRecord = {
        val out = new GenericData.Record(writerSchema)
        out.put("one", one)
        out.put("two", two)
        out
      }
      
      val datumWriter = new GenericDatumWriter[GenericRecord](writerSchema)
      val dataFileWriter = new DataFileWriter[GenericRecord](datumWriter)
      dataFileWriter.create(writerSchema, new java.io.File("/tmp/test.avro"))
      dataFileWriter.append(makeRecord(1, "one"))
      dataFileWriter.append(makeRecord(2, "two"))
      dataFileWriter.append(makeRecord(3, "three"))
      dataFileWriter.close()
      

      Looking at the output with "hexdump -C /tmp/test.avro", we see that the writer schema is embedded in the file, and the record's name is "Writer". To read it back:

      val datumReader = new GenericDatumReader[GenericRecord](writerSchema, readerSchema)
      val dataFileReader = new DataFileReader[GenericRecord](new java.io.File("/tmp/test.avro"), datumReader)
      while (dataFileReader.hasNext) {
        val in = dataFileReader.next()
        println(in, in.getSchema)
      }
      

      The problem is that the above is successful, even though I'm requesting a record with name "Reader".

      If I make structurally incompatible records, for instance by writing with "Writer.two" being an integer and "Reader.two" being a string, it fails to read with org.apache.avro.AvroTypeException (as it should). If I try the above test with an enum type or a fixed type, it does require the writer and reader names to match: record is the only named type for which the name is ignored during schema resolution.

      We're supposed to use aliases to explicitly declare which structurally compatible writer-reader combinations to accept. Because of the above bug, differently named records are accepted regardless of their aliases, but enums and fixed types are not accepted, even if they have the right aliases. This may be a separate bug, or it may be related to the above.

      To make sure that I'm correctly understanding the specification, I tried exactly the same thing in the Python version:

      Unable to find source-code formatter for language: python. Available languages are: actionscript, html, java, javascript, none, sql, xhtml, xml
      import avro.schema
      from avro.datafile import DataFileReader, DataFileWriter
      from avro.io import DatumReader, DatumWriter
      
      writerSchema = avro.schema.parse('{"type": "record", "name": "Writer", "fields": [{"name": "one", "type": "int"}, {"name": "two", "type": "string"}]}')
      readerSchema = avro.schema.parse('{"type": "record", "name": "Reader", "fields": [{"name": "one", "type": "int"}, {"name": "two", "type": "string"}]}')
      
      writer = DataFileWriter(open("/tmp/test2.avro", "w"), DatumWriter(), writerSchema)
      writer.append({"one": 1, "two": "one"})
      writer.append({"one": 2, "two": "two"})
      writer.append({"one": 3, "two": "three"})
      writer.close()
      
      reader = DataFileReader(open("/tmp/test2.avro"), DatumReader(None, readerSchema))
      for datum in reader:
          print datum
      

      The Python code fails in the first read with avro.io.SchemaResolutionException, as it is supposed to. (Interestingly, Python ignores the aliases as well, which I think it's not supposed to do. Since the Java and Python versions both have the same behavior with regard to aliases, I wonder if I'm understanding http://avro.apache.org/docs/1.7.6/spec.html#Aliases correctly.)

        Activity

        Hide
        Jim Pivarski added a comment -

        For me, explicit checking with checkReaderWriterCompatibility is sufficient. But if the default schema check deviates from the specification, that fact should probably be posted. Mentioning it in a section on checkReaderWriterCompatibility would probably suffice for anybody, since anyone can explicitly call this function before iterating over a dataset.

        Show
        Jim Pivarski added a comment - For me, explicit checking with checkReaderWriterCompatibility is sufficient. But if the default schema check deviates from the specification, that fact should probably be posted. Mentioning it in a section on checkReaderWriterCompatibility would probably suffice for anybody, since anyone can explicitly call this function before iterating over a dataset.
        Hide
        Doug Cutting added a comment -

        > When Avro 1.8.x enforces record names [ ... ]

        The question is still one of "if" rather than "when". Incompatible fixes are sometimes not worth making. This has been the case for five years and no one has complained before, so it's not creating a lot of problems.

        Does using explicit checking with checkReaderWriterCompatibility suffice for your case? Perhaps instead of changing the behavior we could just better document current behavior (as you suggest above)?

        What do others think?

        Show
        Doug Cutting added a comment - > When Avro 1.8.x enforces record names [ ... ] The question is still one of "if" rather than "when". Incompatible fixes are sometimes not worth making. This has been the case for five years and no one has complained before, so it's not creating a lot of problems. Does using explicit checking with checkReaderWriterCompatibility suffice for your case? Perhaps instead of changing the behavior we could just better document current behavior (as you suggest above)? What do others think?
        Hide
        Jim Pivarski added a comment -

        Regarding: "Record names are not checked during Java schema resolution. Changing this might break applications that currently work, so a fix should wait until Avro 1.8.0."

        Some applications might inadvertently rely on this behavior, so I understand why you'd want to wait for a backward-incompatible release to make this change. But perhaps there are also applications that intentionally use this feature: for instance, you don't care what record names you're going to receive, but you need them to have a particular subset of fields.

        I'm developing a decision tree processor, and I want to make sure that an input record type has a particular form of predicate and branches to follow if the predicate passes or fails, like this:

        Unable to find source-code formatter for language: json. Available languages are: actionscript, html, java, javascript, none, sql, xhtml, xml
        {"type": "record", "name": "Tree", "fields": [
            {"name": "field", "type": {"type": "enum", "name": "Fields", "symbols": ["one", "two", "three"]}},
            {"name": "comparison", "type": {"type": "enum", "name": "Comparisons", "symbols": ["lessThan", "equalTo"]}},
            {"name": "value", "type": "double"},
            {"name": "pass", "type": ["Tree", {"type": "enum", "name": "Score", "symbols": ["class1", "class2", "class3", "class4"]}]},
            {"name": "fail", "type": ["Tree", "Score"]}
            ]}
        

        (I could relax some of the the enums to be arbitrary strings in cases where I don't know all the fields; this is just for illustration.) The input tree models may have metadata in the form of unspecified fields, such as

        {"name": "numberOfTrainingSamples", "type": "int"}

        , and I'm using Avro's schema resolution to ignore them. They may appear in the writer's schema and are therefore embedded in the model for future reference, but they don't appear in the reader's schema (the decision tree processor) because they're irrelevant to scoring.

        When Avro 1.8.x enforces record names, it could be useful be able to say something like "aliases": ["*"] to accept input records with any name but the correct structure. That way, I'm not putting a restriction on the namespaces of the input records; they can be "com.mycompany.Tree" or "com.yourcompany.Tree" or whatever.

        Should I open a ticket to request "aliases": ["*"]?

        Show
        Jim Pivarski added a comment - Regarding: "Record names are not checked during Java schema resolution. Changing this might break applications that currently work, so a fix should wait until Avro 1.8.0." Some applications might inadvertently rely on this behavior, so I understand why you'd want to wait for a backward-incompatible release to make this change. But perhaps there are also applications that intentionally use this feature: for instance, you don't care what record names you're going to receive, but you need them to have a particular subset of fields. I'm developing a decision tree processor, and I want to make sure that an input record type has a particular form of predicate and branches to follow if the predicate passes or fails, like this: Unable to find source-code formatter for language: json. Available languages are: actionscript, html, java, javascript, none, sql, xhtml, xml { "type" : "record" , "name" : "Tree" , "fields" : [ { "name" : "field" , "type" : { "type" : " enum " , "name" : "Fields" , "symbols" : [ "one" , "two" , "three" ]}}, { "name" : "comparison" , "type" : { "type" : " enum " , "name" : "Comparisons" , "symbols" : [ "lessThan" , "equalTo" ]}}, { "name" : "value" , "type" : " double " }, { "name" : "pass" , "type" : [ "Tree" , { "type" : " enum " , "name" : "Score" , "symbols" : [ "class1" , "class2" , "class3" , "class4" ]}]}, { "name" : "fail" , "type" : [ "Tree" , "Score" ]} ]} (I could relax some of the the enums to be arbitrary strings in cases where I don't know all the fields; this is just for illustration.) The input tree models may have metadata in the form of unspecified fields, such as {"name": "numberOfTrainingSamples", "type": "int"} , and I'm using Avro's schema resolution to ignore them. They may appear in the writer's schema and are therefore embedded in the model for future reference, but they don't appear in the reader's schema (the decision tree processor) because they're irrelevant to scoring. When Avro 1.8.x enforces record names, it could be useful be able to say something like "aliases": ["*"] to accept input records with any name but the correct structure. That way, I'm not putting a restriction on the namespaces of the input records; they can be "com.mycompany.Tree" or "com.yourcompany.Tree" or whatever. Should I open a ticket to request "aliases": ["*"] ?
        Hide
        Jim Pivarski added a comment -

        I've tested aliases, and they completely work--- there's no need to open a new ticket.

        Specifically, I tested enums and fixed, with and without a namespace on the writer, with and without an alias on the reader, with relative and fully-qualified aliases (18 cases). Each case behaved as expected. (I must have gotten my alias-related tests crossed while I was focusing on record names.)

        For the record, I'll post my tests here.

        import org.apache.avro.file.DataFileReader
        import org.apache.avro.file.DataFileWriter
        import org.apache.avro.generic.GenericData
        import org.apache.avro.generic.GenericDatumReader
        import org.apache.avro.generic.GenericDatumWriter
        import org.apache.avro.io.DatumReader
        import org.apache.avro.io.DatumWriter
        import org.apache.avro.Schema
        
        var writerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Writer", "symbols": ["one", "two", "three"]}""")
        var datumWriter = new GenericDatumWriter[GenericData.EnumSymbol](writerSchema)
        var dataFileWriter = new DataFileWriter[GenericData.EnumSymbol](datumWriter)
        dataFileWriter.create(writerSchema, new java.io.File("/tmp/test.avro"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "one"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "two"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "three"))
        dataFileWriter.close()
        
        var readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"]}""")
        var datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        var dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["NOTWRITER"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        writerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Writer", "namespace": "com.wowie", "symbols": ["one", "two", "three"]}""")
        datumWriter = new GenericDatumWriter[GenericData.EnumSymbol](writerSchema)
        dataFileWriter = new DataFileWriter[GenericData.EnumSymbol](datumWriter)
        dataFileWriter.create(writerSchema, new java.io.File("/tmp/test2.avro"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "one"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "two"))
        dataFileWriter.append(new GenericData.EnumSymbol(writerSchema, "three"))
        dataFileWriter.close()
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["com.wowie.Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["com.wowie.NOTWRITER"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "symbols": ["one", "two", "three"], "aliases": ["com.notnamespace.Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "namespace": "com.wowie", "symbols": ["one", "two", "three"], "aliases": ["Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema = (new Schema.Parser).parse("""{"type": "enum", "name": "Reader", "namespace": "com.wowie", "symbols": ["one", "two", "three"], "aliases": ["com.wowie.Writer"]}""")
        datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema)
        dataFileReader = new DataFileReader[GenericData.EnumSymbol](new java.io.File("/tmp/test2.avro"), datumReader)
        while (dataFileReader.hasNext) {
          val in = dataFileReader.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        var writerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Writer", "size": 10}""")
        var datumWriter2 = new GenericDatumWriter[GenericData.Fixed](writerSchema2)
        var dataFileWriter2 = new DataFileWriter[GenericData.Fixed](datumWriter2)
        dataFileWriter2.create(writerSchema2, new java.io.File("/tmp/test3.avro"))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.close()
        
        var readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10}""")
        var datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        var dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test3.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test3.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["NOTWRITER"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test3.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        writerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Writer", "namespace": "com.wowie", "size": 10}""")
        datumWriter2 = new GenericDatumWriter[GenericData.Fixed](writerSchema2)
        dataFileWriter2 = new DataFileWriter[GenericData.Fixed](datumWriter2)
        dataFileWriter2.create(writerSchema2, new java.io.File("/tmp/test4.avro"))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.append(new GenericData.Fixed(writerSchema2, "hellohello".getBytes))
        dataFileWriter2.close()
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["com.wowie.Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["com.wowie.NOTWRITER"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "size": 10, "aliases": ["com.notnamespace.Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // fails (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "namespace": "com.wowie", "size": 10, "aliases": ["Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        readerSchema2 = (new Schema.Parser).parse("""{"type": "fixed", "name": "Reader", "namespace": "com.wowie", "size": 10, "aliases": ["com.wowie.Writer"]}""")
        datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2)
        dataFileReader2 = new DataFileReader[GenericData.Fixed](new java.io.File("/tmp/test4.avro"), datumReader2)
        while (dataFileReader2.hasNext) {
          val in = dataFileReader2.next()
          println(in, in.getSchema)
        }
        // succeeds (good)
        
        Show
        Jim Pivarski added a comment - I've tested aliases, and they completely work--- there's no need to open a new ticket. Specifically, I tested enums and fixed, with and without a namespace on the writer, with and without an alias on the reader, with relative and fully-qualified aliases (18 cases). Each case behaved as expected. (I must have gotten my alias-related tests crossed while I was focusing on record names.) For the record, I'll post my tests here. import org.apache.avro.file.DataFileReader import org.apache.avro.file.DataFileWriter import org.apache.avro. generic .GenericData import org.apache.avro. generic .GenericDatumReader import org.apache.avro. generic .GenericDatumWriter import org.apache.avro.io.DatumReader import org.apache.avro.io.DatumWriter import org.apache.avro.Schema var writerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Writer ", " symbols ": [" one ", " two ", " three "]}" "") var datumWriter = new GenericDatumWriter[GenericData.EnumSymbol](writerSchema) var dataFileWriter = new DataFileWriter[GenericData.EnumSymbol](datumWriter) dataFileWriter.create(writerSchema, new java.io.File( "/tmp/test.avro" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "one" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "two" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "three" )) dataFileWriter.close() var readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "]}" "") var datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) var dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // fails (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // succeeds (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" NOTWRITER "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // fails (good) writerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Writer ", " namespace ": " com.wowie ", " symbols ": [" one ", " two ", " three "]}" "") datumWriter = new GenericDatumWriter[GenericData.EnumSymbol](writerSchema) dataFileWriter = new DataFileWriter[GenericData.EnumSymbol](datumWriter) dataFileWriter.create(writerSchema, new java.io.File( "/tmp/test2.avro" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "one" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "two" )) dataFileWriter.append( new GenericData.EnumSymbol(writerSchema, "three" )) dataFileWriter.close() readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // fails (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" com.wowie.Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // succeeds (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" com.wowie.NOTWRITER "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // fails (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " symbols ": [" one ", " two ", " three "], " aliases ": [" com.notnamespace.Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // fails (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " namespace ": " com.wowie ", " symbols ": [" one ", " two ", " three "], " aliases ": [" Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // succeeds (good) readerSchema = ( new Schema.Parser).parse( """{" type ": " enum ", " name ": " Reader ", " namespace ": " com.wowie ", " symbols ": [" one ", " two ", " three "], " aliases ": [" com.wowie.Writer "]}" "") datumReader = new GenericDatumReader[GenericData.EnumSymbol](writerSchema, readerSchema) dataFileReader = new DataFileReader[GenericData.EnumSymbol]( new java.io.File( "/tmp/test2.avro" ), datumReader) while (dataFileReader.hasNext) { val in = dataFileReader.next() println(in, in.getSchema) } // succeeds (good) var writerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Writer ", " size ": 10}" "") var datumWriter2 = new GenericDatumWriter[GenericData.Fixed](writerSchema2) var dataFileWriter2 = new DataFileWriter[GenericData.Fixed](datumWriter2) dataFileWriter2.create(writerSchema2, new java.io.File( "/tmp/test3.avro" )) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.close() var readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10}" "") var datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) var dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test3.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // fails (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test3.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // succeeds (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" NOTWRITER "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test3.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // fails (good) writerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Writer ", " namespace ": " com.wowie ", " size ": 10}" "") datumWriter2 = new GenericDatumWriter[GenericData.Fixed](writerSchema2) dataFileWriter2 = new DataFileWriter[GenericData.Fixed](datumWriter2) dataFileWriter2.create(writerSchema2, new java.io.File( "/tmp/test4.avro" )) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.append( new GenericData.Fixed(writerSchema2, "hellohello" .getBytes)) dataFileWriter2.close() readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // fails (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" com.wowie.Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // succeeds (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" com.wowie.NOTWRITER "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // fails (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " size ": 10, " aliases ": [" com.notnamespace.Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // fails (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " namespace ": " com.wowie ", " size ": 10, " aliases ": [" Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // succeeds (good) readerSchema2 = ( new Schema.Parser).parse( """{" type ": " fixed ", " name ": " Reader ", " namespace ": " com.wowie ", " size ": 10, " aliases ": [" com.wowie.Writer "]}" "") datumReader2 = new GenericDatumReader[GenericData.Fixed](writerSchema2, readerSchema2) dataFileReader2 = new DataFileReader[GenericData.Fixed]( new java.io.File( "/tmp/test4.avro" ), datumReader2) while (dataFileReader2.hasNext) { val in = dataFileReader2.next() println(in, in.getSchema) } // succeeds (good)
        Hide
        Jim Pivarski added a comment -

        Hi! Working through your bullet points here...

        Thanks for pointing me to org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility (from AVRO-1409): it's exactly what I was looking for. Something like that ought to be briefly mentioned on the Getting Started (Java) page.

        My examples work in this per-schema validation, including names and including aliases, but my bug report about the per-instance validation stands.

        Does decoding always go through the following sequence: (1) read embedded schema, (2) loop through data instances? If so, then the problem of a too-permissive instance validation can be solved by doing a one-time schema validation at the beginning of reading, using checkReaderWriterCompatibility, which does the full tests. In fact, that may be preferable to doing redundant checks on every instance.

        I'm going to look into aliases in more detail, and if I find what looks like a bug, I'll file another bug report. I won't bother checking the Python version because of its lack of support.

        Show
        Jim Pivarski added a comment - Hi! Working through your bullet points here... Thanks for pointing me to org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility (from AVRO-1409 ): it's exactly what I was looking for. Something like that ought to be briefly mentioned on the Getting Started (Java) page. My examples work in this per-schema validation, including names and including aliases, but my bug report about the per-instance validation stands. Does decoding always go through the following sequence: (1) read embedded schema, (2) loop through data instances? If so, then the problem of a too-permissive instance validation can be solved by doing a one-time schema validation at the beginning of reading, using checkReaderWriterCompatibility, which does the full tests. In fact, that may be preferable to doing redundant checks on every instance. I'm going to look into aliases in more detail, and if I find what looks like a bug, I'll file another bug report. I won't bother checking the Python version because of its lack of support.
        Hide
        Doug Cutting added a comment -

        A few separate issues are described above.

        • Record names are not checked during Java schema resolution. Changing this might break applications that currently work, so a fix should wait until Avro 1.8.0. In the meantime, AVRO-1409 permits applications to explicitly check compatibility (including name checks). This can be useful, since many schema resolution problems are currently only detected as data is read rather than when a data file is opened or earlier.
        • Aliases in Java do not work in enums or fixed. There are tests for aliases in enum and fixed, but perhaps these are not sufficient. (https://svn.apache.org/viewvc?view=revision&revision=r986455) If you can provide another test case that illustrates a failure we can probably fix this in a bugfix release (e.g., 1.7.7). Please file a separate issue for this.
        • Python does not implement aliases. Aliases are an optional feature that an implementation may support. Java may be the only implementation that currently supports aliases. There is an existing feature request to implement aliases in Python (AVRO-1303).
        Show
        Doug Cutting added a comment - A few separate issues are described above. Record names are not checked during Java schema resolution. Changing this might break applications that currently work, so a fix should wait until Avro 1.8.0. In the meantime, AVRO-1409 permits applications to explicitly check compatibility (including name checks). This can be useful, since many schema resolution problems are currently only detected as data is read rather than when a data file is opened or earlier. Aliases in Java do not work in enums or fixed. There are tests for aliases in enum and fixed, but perhaps these are not sufficient. ( https://svn.apache.org/viewvc?view=revision&revision=r986455 ) If you can provide another test case that illustrates a failure we can probably fix this in a bugfix release (e.g., 1.7.7). Please file a separate issue for this. Python does not implement aliases. Aliases are an optional feature that an implementation may support. Java may be the only implementation that currently supports aliases. There is an existing feature request to implement aliases in Python ( AVRO-1303 ).

          People

          • Assignee:
            Unassigned
            Reporter:
            Jim Pivarski
          • Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

            • Created:
              Updated:

              Development