Avro
  1. Avro
  2. AVRO-127

Avro should support multiple schemas from the same AVRO file

    Details

    • Type: New Feature New Feature
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.2.0
    • Component/s: java
    • Labels:
      None
    • Environment:

      all systems

      Description

      in our application, we often have to merge together all the day's data into one daily file,
      data file schemas must be changing within the day, so we have different schemas within the same file.

      AVRO should support multiple schemas within the same file, ideally different schemas can be parsed out
      to different children classes of the same schema class.

        Activity

        Transition Time In Source Status Execution Times Last Executer Last Execution Date
        Open Open Resolved Resolved
        7h 41m 1 Yang Yang 29/Sep/09 01:23
        Resolved Resolved Closed Closed
        21d 17h 45m 1 Doug Cutting 20/Oct/09 19:08
        Doug Cutting made changes -
        Component/s java [ 12312780 ]
        Doug Cutting made changes -
        Status Resolved [ 5 ] Closed [ 6 ]
        Doug Cutting made changes -
        Fix Version/s 1.2.0 [ 12314242 ]
        Yang Yang made changes -
        Field Original Value New Value
        Status Open [ 1 ] Resolved [ 5 ]
        Resolution Fixed [ 1 ]
        Hide
        Yang Yang added a comment -

        got it, thanks, closing this

        Show
        Yang Yang added a comment - got it, thanks, closing this
        Hide
        Doug Cutting added a comment -

        This technique relies on the way that unions are encoded. In general, if new elements are added to the end of a union schema then data written with this schema before these were added can still be read. So you can do the same thing with GenericDatumWriter: just create the new extended union schema, then either use DatumWriter#setSchema() to install it or create a new DatumWriter with the new extended schema.

        Note that you should store the union schema somewhere with the data, so that you can be guaranteed to be able to read it later. Avro's data file format takes care of this for you.

        > I normally use GenericDatumWriter so that I can use it directly on HDFS ,instead of facing a file
        > (the hadoop FileInput/FileOutputFormat API gives me a DataInputStream to work with, not a file)

        Avro's DataFileWriter constructor accepts an OutputStream, and, with a bit of code you can implement Avro's SeekableInput interface so that you can read an Avro data file from HDFS:

        public class SeekableHadoopInput extends FilterInputStream implements SeekableInput {
          public SeekableHadoopInput(FSDataInputStream in, long length) { 
            super(in);
            this.length = length;
          }
          public long length() { return length; }
          public void seek(long pos) { ((FSDataInputStream)in).seek(pos); }
          public long tell() { return ((FSDataInputStream)in).getPos(); }
        }
        

        Alternately, in Hadoop 0.21 you can store Avro data in SequenceFiles. But we do really need to create InputFormat and OutputFormat's for Avro's data file...

        Show
        Doug Cutting added a comment - This technique relies on the way that unions are encoded. In general, if new elements are added to the end of a union schema then data written with this schema before these were added can still be read. So you can do the same thing with GenericDatumWriter: just create the new extended union schema, then either use DatumWriter#setSchema() to install it or create a new DatumWriter with the new extended schema. Note that you should store the union schema somewhere with the data, so that you can be guaranteed to be able to read it later. Avro's data file format takes care of this for you. > I normally use GenericDatumWriter so that I can use it directly on HDFS ,instead of facing a file > (the hadoop FileInput/FileOutputFormat API gives me a DataInputStream to work with, not a file) Avro's DataFileWriter constructor accepts an OutputStream, and, with a bit of code you can implement Avro's SeekableInput interface so that you can read an Avro data file from HDFS: public class SeekableHadoopInput extends FilterInputStream implements SeekableInput { public SeekableHadoopInput(FSDataInputStream in, long length) { super(in); this.length = length; } public long length() { return length; } public void seek(long pos) { ((FSDataInputStream)in).seek(pos); } public long tell() { return ((FSDataInputStream)in).getPos(); } } Alternately, in Hadoop 0.21 you can store Avro data in SequenceFiles. But we do really need to create InputFormat and OutputFormat's for Avro's data file...
        Hide
        Yang Yang added a comment -

        yes, this is exactly what we want.

        the schema change frequency is fine, we will have about at most 1--2 changes each month, so about 1 or 2 daily files out of every 30 files will have
        more than 1 schema in them.

        having a new constructor isn't so urgently needed, the above "empty union--->addSchema() " approach is good enough.

        but there is indeed a problem: I normally use GenericDatumWriter so that I can use it directly on HDFS , instead of facing a file (the hadoop FileInput/FileOutputFormat
        API gives me a DataInputStream to work with, not a file), it seems that GenericDatumWriter does not have addSchema() method. does that need to be copied over
        to GenericDatumWriter ?

        Thanks
        Yang

        Show
        Yang Yang added a comment - yes, this is exactly what we want. the schema change frequency is fine, we will have about at most 1--2 changes each month, so about 1 or 2 daily files out of every 30 files will have more than 1 schema in them. having a new constructor isn't so urgently needed, the above "empty union--->addSchema() " approach is good enough. but there is indeed a problem: I normally use GenericDatumWriter so that I can use it directly on HDFS , instead of facing a file (the hadoop FileInput/FileOutputFormat API gives me a DataInputStream to work with, not a file), it seems that GenericDatumWriter does not have addSchema() method. does that need to be copied over to GenericDatumWriter ? Thanks Yang
        Hide
        Doug Cutting added a comment -

        In Avro 1.1 we added DataFileWriter#addSchema(Schema):

        http://hadoop.apache.org/avro/docs/current/api/java/org/apache/avro/file/DataFileWriter.html#addSchema%28org.apache.avro.Schema%29

        This only works if the top-level schema is a union. It lets you add clauses to that union as you write the file. So, for example, you might do something like:

        DataFileWriter<Object> out = 
          new DataFileWriter<Object>(Schema.createUnion(new ArrayList()[])),
           new FileOutputStream("foo"), new GenericDatumWriter());
        ...
        out.addSchema(x.getSchema());
        out.write(x);
        ...
        out.addSchema(y.getSchema());
        out.write(y)
        ...
        

        Does that achieve what you want? This should work with specific or reflect data too.

        Note that you should only add a given schema once. Also, if you have a large number of schemas (>10?) per file then performance may lag a bit, since it scans the union linearly each time an entry is appended. But that could easily be optimized if it became a problem.

        I am tempted to make this the default for DataFileWriter: that we add a constructor that does not include a schema, in which case an empty union is used. Also, we could then automatically add schemas to the union if they're not already present. Would that be useful?

        Show
        Doug Cutting added a comment - In Avro 1.1 we added DataFileWriter#addSchema(Schema): http://hadoop.apache.org/avro/docs/current/api/java/org/apache/avro/file/DataFileWriter.html#addSchema%28org.apache.avro.Schema%29 This only works if the top-level schema is a union. It lets you add clauses to that union as you write the file. So, for example, you might do something like: DataFileWriter<Object> out = new DataFileWriter<Object>(Schema.createUnion(new ArrayList()[])), new FileOutputStream("foo"), new GenericDatumWriter()); ... out.addSchema(x.getSchema()); out.write(x); ... out.addSchema(y.getSchema()); out.write(y) ... Does that achieve what you want? This should work with specific or reflect data too. Note that you should only add a given schema once. Also, if you have a large number of schemas (>10?) per file then performance may lag a bit, since it scans the union linearly each time an entry is appended. But that could easily be optimized if it became a problem. I am tempted to make this the default for DataFileWriter: that we add a constructor that does not include a schema, in which case an empty union is used. Also, we could then automatically add schemas to the union if they're not already present. Would that be useful?
        Yang Yang created issue -

          People

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

            Dates

            • Created:
              Updated:
              Resolved:

              Development