A JIRA is filed -- AVRO-792.

From: [email protected]
To: [email protected]
Date: Wed, 30 Mar 2011 14:15:07 -0700
Subject: Re: map reduce job for avro 1.5 broken




Can you file a JIRA so we can track this issue there?
I have tried a few things to reproduce it on my end with no success, and will 
have some time to look into it in a couple days.
Thanks!
On 3/30/11 1:04 PM, "ey-chih chow" <[email protected]> wrote:

Is there anybody can tell us why the map reduce job failed?  We have switched 
back to Avro 1.4.0 and everything works fine again, but for us to continue 
using Avro in the future, we have to be able to upgrade to Avro 1.5.  We tried 
1.5 in our debugging environment and everything works fine, but when we switch 
to a real small cluster, the MR job always throws exceptions for some reducers, 
as indicated in the previous messages.  I saw many avro code changes in 
serialization/de-serialization between 1.4 and 1.5.  Can somebody let me know 
how I can find the problem?  Thanks.
Ey-Chih Chow     

From: [email protected]
To: [email protected]
Subject: RE: map reduce job for avro 1.5 broken
Date: Mon, 28 Mar 2011 17:47:10 -0700




The signature of our map() is:
public void map(Utf8 input, AvroCollector<Pair<Utf8, GenericRecord>> collector, 
Reporter reporter) throws IOException;
and the correspond reduce() Is:
public void reduce(Utf8 key, Iterable<GenericRecord> values, 
AvroCollector<GenericRecord> collector, Reporter reporter) throws IOException;
The schema for GenericRecord are the same.

For this map/reduce job, we have 23 reducers.  Four of them succeeded and the 
rest failed because of this exception.



From: [email protected]
To: [email protected]
Date: Mon, 28 Mar 2011 17:10:36 -0700
Subject: Re: map reduce job for avro 1.5 broken

I was able to serialize and read back the following schema with 1.5 using 
GenericDatumWriter and GenericDatumReader:
{"name":"foo", "type":"record", "fields":[  {"name":"mymap", "type":[    
{"type":"map", "values":["int","long","float","string"]},    "null"]  }]}
Your traces below look like they are in the resolver.  Are your writer and 
reader schemas the same?  If it is related to schema resolution we need both 
versions of the schema — as it was written ('writer' schema) and what it is 
being resolved to ('reader' schema).
On 3/28/11 2:45 PM, "ey-chih chow" <[email protected]> wrote:

Hi,
We have an avro map/reduce job used to be working with avro 1.4, but broken 
with avro 1.5 when the reducer tried to do de-serilization.  By looking at the 
trace, it looked like the reducer was broken when trying to resolve 'union' of 
a 'map' definition in our avdl schema.  We have three fields in our schema 
relating to this.  These are:
        union   {map <union {int,long,float,string}>, null} evpl;       union   
{map <union {int,long,float,string}>, null} plst;       union   {map <union 
{int,long,float,string}>, null} change;

Can anybody let me know if this is a 1.5 bug?  The stack trace was as follows:
java.lang.ArrayIndexOutOfBoundsException: -1576799025   at 
org.apache.avro.io.parsing.Symbol$Alternative.getSymbol(Symbol.java:364)     at 
org.apache.avro.io.ResolvingDecoder.doAction(ResolvingDecoder.java:229)      at 
org.apache.avro.io.parsing.Parser.advance(Parser.java:88)    at 
org.apache.avro.io.ResolvingDecoder.readIndex(ResolvingDecoder.java:206)     at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:142) at 
org.apache.avro.generic.GenericDatumReader.readMap(GenericDatumReader.java:232) 
     at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:141) at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:142) at 
org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:166)
   at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:138) at 
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:129) at 
org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer.deserialize(AvroSerialization.java:86)
      at 
org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer.deserialize(AvroSerialization.java:68)
      at 
org.apache.hadoop.mapred.Task$ValuesIterator.readNextValue(Task.java:1136)   at 
org.apache.hadoop.mapred.Task$ValuesIterator.next(Task.java:1076)    at 
org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator.moveToNext(ReduceTask.java:246)
     at 
org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator.next(ReduceTask.java:242)
   at 
org.apache.avro.mapred.HadoopReducerBase$ReduceIterable.next(HadoopReducerBase.java:47)
      at 
com.ngmoco.ngpipes.etl.NgEventETLReducer.reduce(NgEventETLReducer.java:46)   at 
com.ngmoco.ngpipes.etl.NgEventETLReducer.reduce(NgEventETLReducer.java:1)    at 
org.apache.avro.mapred.HadoopReducerBase.reduce(HadoopReducerBase.java:60)   at 
org.apache.avro.mapred.HadoopReducerBase.reduce(HadoopReducerBase.java:30)   at 
org.apache.hadoop.mapred.ReduceTask.runOldReducer(ReduceTask.java:468)       at 
org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:416) at 
org.apache.hadoop.mapred.Child$4.run(Child.java:240) at 
java.security.AccessController.doPrivileged(Native Method)   at 
javax.security.auth.Subject.doAs(Subject.java:396)   at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1115)
 at org.apache.hadoop.mapred.Child.main(Child.java:234)
Thanks.
Ey-Chih Chow                                                                    
                                                  

Reply via email to