avro-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Scott Carey <sc...@richrelevance.com>
Subject Re: Question on writing/reading file with different schema
Date Thu, 15 Apr 2010 17:15:18 GMT
It appears there is a bug in the ResolvingDecoder when the actual schema has trailing fields
not in the expected schema.  I have not had time to track it down.  I filed a JIRA ticket:
https://issues.apache.org/jira/browse/AVRO-517.

I have a suggested work-around.  You probably don't want to explicitly use a different reader
schema than the file.  The primary use case of the schema resolving is for schema evolution
and migration.  Most of the time a single version of an application will want to use a single
schema to represent the data.
If you simply want to read 2 of 3 fields, read 2 of three fields from the full schema -- don't
define a schema with only 2 of the three fields.
Every client can use the full "Person" schema, but wrapper classes or helper methods can read
the subset of the fields they want to.

In the example below, browseAge() and browseName() can use personSchema, there is no need
to create or manage the other two schemas.  I am not sure if that applies to your real-world
usage, but it likely does.

-Scott

On Apr 13, 2010, at 12:04 AM, Lurga wrote:

> Hi,
> 27
> 20
> 31
> Dante Hicks	
> Exception in thread "main" org.apache.avro.AvroRuntimeException: java.io.EOFException
> 	at org.apache.avro.file.DataFileStream.next(DataFileStream.java:184)
> 	at cn.znest.test.avro.AddressBook.browseName(AddressBook.java:91)
> 	at cn.znest.test.avro.AddressBook.main(AddressBook.java:43)
> Caused by: java.io.EOFException
> 	at org.apache.avro.io.BinaryDecoder.readInt(BinaryDecoder.java:163)
> 	at org.apache.avro.io.BinaryDecoder.readString(BinaryDecoder.java:262)
> 	at org.apache.avro.io.ValidatingDecoder.readString(ValidatingDecoder.java:93)
> 	at org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:277)
> 	at org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:271)
> 	at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:83)
> 	at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:105)
> 	at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:77)
> 	at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:70)
> 	at org.apache.avro.file.DataFileStream.next(DataFileStream.java:195)
> 	at org.apache.avro.file.DataFileStream.next(DataFileStream.java:182)
> 	... 2 more
> 
> My code is below. In this example, I create three record: person(3 fileds: First Last
Age), age(Age), extract(First Last). The record "age" has the last filed of "Person", so AddressBook.browseAge()
will be executed successfully. But the record "extract" does not have the last filed of "Person",
so executing AddressBook.browseName() will cause an exception.  
> In avro/c, read_record (datum_read.c) loops every write_schema fileds. 
> In avro/java, GenericDatumReader.readRecord  loops every read_schema fileds. I think
that's the point.
> 
> import java.io.File;
> import java.io.IOException;
> 
> import org.apache.avro.Schema;
> 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.GenericData.Record;
> import org.apache.avro.util.Utf8;
> 
> public class AddressBook {
> 	String fileName = "AddressBook.db";
> 	String prefix = "{\"type\":\"record\",\"name\": \"Person\",\"fields\":[";
> 	String suffix = "]}";
> 	String fieldFirst = "{\"name\":\"First\",\"type\":\"string\"}";
> 	String fieldLast = "{\"name\":\"Last\",\"type\":\"string\"}";
> 	String fieldAge = "{\"name\":\"Age\",\"type\":\"int\"}";
> 	Schema personSchema = Schema.parse(prefix + fieldFirst + "," + fieldLast + ","  + fieldAge
+ suffix);
> 	Schema ageSchema = Schema.parse(prefix + fieldAge + suffix);
> 	Schema extractSchema = Schema.parse(prefix + fieldFirst + "," + fieldLast + suffix);
> 	/**
> 	 * @param args
> 	 * @throws IOException
> 	 */
> 	public static void main(String[] args) throws IOException {
> 		AddressBook ab = new AddressBook();
> 		ab.init();
> 		ab.browseAge();
> 		ab.browseName();
> 	}
> 
> 	public void init() throws IOException {		
> 		DataFileWriter<Record> writer = new DataFileWriter<Record>(
> 				new GenericDatumWriter<Record>(personSchema)).create(
> 						personSchema, new File(fileName));
> 		try {
> 			writer.append(createPerson("Dante", "Hicks", 27));
> 			writer.append(createPerson("Randal", "Graves", 20));
> 			writer.append(createPerson("Steve", "Jobs", 31));
> 		} finally {
> 			writer.close();
> 		}
> 	}
> 	
> 	private Record createPerson(String first, String last, int age) {
> 		Record person = new GenericData.Record(personSchema);
> 		person.put("First", new Utf8(first));
> 		person.put("Last", new Utf8(last));
> 		person.put("Age", age);
> 		return person;
> 	}
> 	
> 	public void browseAge() throws IOException {		
> 		GenericDatumReader<Record> dr = new GenericDatumReader<Record>();
> 		dr.setExpected(ageSchema);
> 		DataFileReader<Record> reader = new DataFileReader<Record>(new File(
> 		  fileName), dr);
> 		
> 		try {
> 			while (reader.hasNext()) {
> 				Record person = reader.next();
> 				System.out.println(person.get("Age").toString());
> 			}
> 		} finally {
> 			reader.close();
> 		}
> 	}
> 	
> 	public void browseName() throws IOException {		
> 		GenericDatumReader<Record> dr = new GenericDatumReader<Record>();
> 		dr.setExpected(extractSchema);
> 		DataFileReader<Record> reader = new DataFileReader<Record>(new File(
> 		  fileName), dr);
> 		
> 		try {
> 			while (reader.hasNext()) {
> 				Record person = reader.next();
> 				System.out.println(person.get("First").toString() + " " + person.get("Last").toString()
+ "\t");
> 			}
> 		} finally {
> 			reader.close();
> 		}
> 	}
> }
> 
> 
> 2010-04-13 
> Lurga            
> 
> 发件人: Scott Carey 
> 发送时间: 2010-04-13  10:55:41 
> 收件人: avro-user@hadoop.apache.org 
> 抄送: 
> 主题: Re: Question on writing/reading file with different schema 
> 
> On Apr 12, 2010, at 7:42 PM, Scott Carey wrote:
>> Try something like:
>> 
>> DatumReader dr = new GenericDatumReader();
>> dr.setExpected(extractSchema);
>> DataFileReader<Record> reader = new DataFileReader<Record>(new File(
>> fileName), dr);
>> 
> If the above still fails, please provide the full stack trace that results.
>> 
>>> Regards,
>>> 
>>> 2010-04-13 
>>> 
>>> 
>>> 
>>> Lurga 
>>> 
>> 

Mime
View raw message