[
https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12913325#action_12913325
]
Ron Bodkin commented on AVRO-669:
---------------------------------
That makeWriter method was a hold-over from an earlier attempt at allowing the
use of either reflect or specific. If we can let it always use reflect that
would be a lot better (and I'd just eliminate makeWriter). Likewise for that
constructor flag - hopefully something to just drop.
To fit the approach of converting Utf8 to CharSequence there would also be a
need to have a similar change in any of the Avro-generated classes, e.g., if I
try to read an array of strings with reflection using Avro-generated code I get
this exception:
java.lang.ClassCastException: [Ljava.lang.String; cannot be cast to
java.util.List
This got generated as
public java.util.List<java.lang.CharSequence> key;
The back trace is:
at org.apache.avro.generic.GenericData.setField(GenericData.java:377)
at org.apache.avro.reflect.ReflectData.setField(ReflectData.java:79)
at
org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:149)
at
org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:121)
at
org.apache.avro.generic.GenericDatumReader.readArray(GenericDatumReader.java:178)
The problem here is that the system can't detect the type of null fields. Since
it's possible to have fields that should be null, we can't just initialize an
empty collection. Instead, it looks to me like the generated schema for a
generated class needs to emit the right hints in the form of these schema
properties:
static final String CLASS_PROP = "java-class";
static final String ELEMENT_PROP = "java-element-class";
> 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.