[ 
https://issues.apache.org/jira/browse/AVRO-1440?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13872382#comment-13872382
 ] 

Doug Cutting commented on AVRO-1440:
------------------------------------

Avro requires that the reader have the writer's schema as well.  In data files 
the writer's schema is stored in the header.  In RPC this is arranged through 
the handshake.  I see no evidence in your examples that the writer's schema is 
provided.  You are attempting to read data providing only the reader's schema.

To make this work you will need to use something like the following to create 
your DatumReader:

{code}
new GenericDatumReader<Object>(userLessSchema, userMoreSchema);
{code}

Moreover, if these two schemas have different names, you'll need to provide the 
old name as an alias, e.g.:

{code}
... "name":"UserMore", "aliases":["UserLess"] ...
{code}

Lastly, when you add a new field that's not present in data you read you must 
specify a default value:

{code}
{"name":"favoritecolor", "type":"string", "default":"blue"}
{code}


> the  problem  of resolving schema 
> ----------------------------------
>
>                 Key: AVRO-1440
>                 URL: https://issues.apache.org/jira/browse/AVRO-1440
>             Project: Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.7.5
>         Environment: jdk1.7.0_45
>            Reporter: nivance
>              Labels: patch
>
> In Apache Avro™ 1.7.5 Documentation, have a sentence like this:
> {quote}Since both client and server both have the other's full schema, 
> correspondence between same named fields, missing fields, extra fields, etc. 
> can all be easily resolved.{quote}
> *But if client has two fields and server has three fields, the server can't 
> resolve.*
> *Client avsc:*
> {"type":"record",
> "name":"UserLess",
> "namespace":"com.joyveb.test.bean",
> "fields":[{"name":"name","type":"string"},{"name":"favoritenumber","type":"int"}]}
> *serialize codes:*
> ByteArrayOutputStream bao = new ByteArrayOutputStream();
> GenericDatumWriter<Object> w = new 
> GenericDatumWriter<Object>(userLess.getSchema());
> Encoder e = EncoderFactory.get().binaryEncoder(bao, null);
> w.write(object, e);
> e.flush();
> ByteBuffer.wrap(bao.toByteArray());
> *Server avsc:*
> {"type":"record",
> "name":"UserMore",
> "namespace":"com.joyveb.test.bean",
> "fields":[{"name":"name","type":"string"},{"name":"favoritenumber","type":"int"},{"name":"favoritecolor","type":"string"}]}
> *deserialize codes:*
> BinaryDecoder in = DecoderFactory.get().binaryDecoder(bytes, null);
> Object result = new 
> GenericDatumReader<Object>(userMore.getSchema()).read(null, in);
> *After run the program, following exception comes out.*
> Exception in thread "main" java.io.EOFException
>       at org.apache.avro.io.BinaryDecoder.ensureBounds(BinaryDecoder.java:473)
>       at org.apache.avro.io.BinaryDecoder.readInt(BinaryDecoder.java:128)
>       at org.apache.avro.io.BinaryDecoder.readString(BinaryDecoder.java:259)
>       at 
> org.apache.avro.io.ValidatingDecoder.readString(ValidatingDecoder.java:107)
>       at 
> org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:363)
>       at 
> org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:355)
>       at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:157)
>       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 com.joyveb.dbpapi.AvroByteUtil.deserialize(AvroByteUtil.java:81)
>       at 
> com.joyveb.dbpapi.test.avro.SchemaByteTest.less2more(SchemaByteTest.java:34)
>       at 
> com.joyveb.dbpapi.test.avro.SchemaByteTest.main(SchemaByteTest.java:15)



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to