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

Doug Cutting commented on AVRO-669:
-----------------------------------

> What do you think is the right way to handle this inconsistency?

In TestWordCount,.java, we should change all non-constructor references to Utf8 
to CharSequence, the common interface between String and Utf8.

Looking at your patch, you've added a few features without explanation.  You've 
added an AvroSerialization#makeWriter() method.  This might be reasonable, but 
since it's protected and will appear in javadoc, it deserves a javadoc comment. 
 To AvroRecordReader you add a new constructor with a flag, indicating whether 
reflection's to be used.  Should we rather be consistent here in how we specify 
this?

An early version of this API had a job parameter, avro.input.api, 
avro.mapout.api and avro.output.api or somesuch, that could have values 
"reflect", "specific", or "generic".  Perhaps we should revive that approach?  
The default would be "reflect", and mapout would default to output.  Thoughts?


> Avro Mapreduce Doesn't Work With Reflect Schemas
> ------------------------------------------------
>
>                 Key: AVRO-669
>                 URL: https://issues.apache.org/jira/browse/AVRO-669
>             Project: Avro
>          Issue Type: Bug
>            Reporter: Ron Bodkin
>            Assignee: Doug Cutting
>             Fix For: 1.5.0
>
>         Attachments: AVRO-669.patch, AVRO-669.patch.2
>
>
> I'm trying to get the Avro trunk code (from Subversion) to work with a simple 
> example of a reflection-defined schema, using a class I created. I use a 
> ReflectDatumWriter to write a set of records to a file, e.g.,
>         DatumWriter writer = new ReflectDatumWriter(Record.class);
>         DataFileWriter file = new DataFileWriter(writer);
> However, when I try to read that data in using an AvroMapper it fails with an 
> exception as shown below. It turns out that the mapreduce implementation 
> hard-codes a dependence on SpecificDatum readers and writers. 
> I've tested switching to use ReflectDatum instead in five places to try to 
> get it to work for an end-to-end reflect data example:
> AvroFileInputFormat
> AvroFileOutputFormat
> AvroSerialization (getDeserializer and getSerializer)
> AvroKeyComparator
> However, switching to use reflection for AvroKeyComparator doesn't work:
> java.lang.UnsupportedOperationException
>       at org.apache.avro.reflect.ReflectData.compare(ReflectData.java:427)
>       at 
> org.apache.avro.mapred.AvroKeyComparator.compare(AvroKeyComparator.java:46)
> It should be possible to implement compare on reflect data (just like 
> GenericData's implementation but use the field name instead (or better yet a 
> cached java.lang.reflect.Field)...
> Original exception:
> java.lang.ClassCastException: tba.mr.sample.avro.Record cannot be cast to 
> org.apache.avro.generic.IndexedRecord
>       at 
> org.apache.avro.generic.GenericDatumReader.setField(GenericDatumReader.java:152)
>       at 
> org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:142)
>       at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:114)
>       at 
> org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:105)
>       at org.apache.avro.file.DataFileStream.next(DataFileStream.java:198)
>       at 
> org.apache.avro.mapred.AvroRecordReader.next(AvroRecordReader.java:63)
>       at 
> org.apache.avro.mapred.AvroRecordReader.next(AvroRecordReader.java:33)
>       at 
> org.apache.hadoop.mapred.MapTask$TrackedRecordReader.moveToNext(MapTask.java:192)
>       at 
> org.apache.hadoop.mapred.MapTask$TrackedRecordReader.next(MapTask.java:176)
>       at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:48)
>       at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358)
>       at org.apache.hadoop.mapred.MapTask.run(MapTask.java:307)
>       at org.apache.hadoop.mapred.Child.main(Child.java:170)

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to