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

Schema resolution does not check record names

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 1.7.6
    • None
    • java
    • None
    • 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:

      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.)

      Attachments

        Activity

          People

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

            Dates

              Created:
              Updated: