You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Ron Bodkin (JIRA)" <ji...@apache.org> on 2010/09/17 07:48:32 UTC

[jira] Created: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

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
             Fix For: 1.5.0


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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Spike Gronim (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12920376#action_12920376 ] 

Spike Gronim commented on AVRO-669:
-----------------------------------

I have also experienced this issue. In my case I could work around it by disabling the Combiner.

> 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.


[jira] Updated: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated AVRO-669:
------------------------------

    Attachment: AVRO-669.patch

Here's a patch that implements ReflectData.compare() for all types but Object[] and byte[].  It moves the getField() and setField() methods from the reader/writers to GenericData, so that the generic implementation of record comparison can be shared.  This also eliminates a little duplicate code.

It doesn't add full end-to-end MapReduce tests for reflect-based data, so I have not verified whether that works yet, but it should address the proximal cause.

AVRO-638 is related.  When that's complete, it should be possible to switch the mapred code to always use ReflectDatumReader, ReflectDatumWriter and ReflectData, since these should then work for specific and generic data as well.

Please tell me if this helps.

> 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
>             Fix For: 1.5.0
>
>         Attachments: AVRO-669.patch
>
>
> 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.


[jira] Assigned: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting reassigned AVRO-669:
---------------------------------

    Assignee: Doug Cutting

> 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
>
>
> 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Ron Bodkin (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12910856#action_12910856 ] 

Ron Bodkin commented on AVRO-669:
---------------------------------

Thanks Doug, that does help. With those changes (and a few more that I'll attach as a patch to be applied in addition) I was able to run my sample code.

> 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
>
>
> 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.


[jira] Updated: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Ron Bodkin (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ron Bodkin updated AVRO-669:
----------------------------

    Attachment: AVRO-669.patch.2

Additional patch, to applied after AVRO-669.patch, to allow reflect data to be used with mapreduce.

I haven't got the map/reduce tests for avro working in my development environment, so they still needs to be verified and extended to test reflect data as well.


> 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Ron Bodkin (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Ron Bodkin (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12911082#action_12911082 ] 

Ron Bodkin commented on AVRO-669:
---------------------------------

This change does break the word count test:

  [junit] java.lang.ClassCastException: java.lang.String cannot be cast to org.apache.avro.util.Utf8
    [junit] 	at org.apache.avro.mapred.TestWordCount$MapImpl.map(TestWordCount.java:43)
    [junit] 	at org.apache.avro.mapred.HadoopMapper.map(HadoopMapper.java:80)
    [junit] 	at org.apache.avro.mapred.HadoopMapper.map(HadoopMapper.java:34)
    [junit] 	at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50)
    [junit] 	at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358)
    [junit] 	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:307)
    [junit] 	at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:177)

This was because ReflectDatumReader.readString returns a String, whereas GenericDatumReader.readString returns a Utf8.

I'm tested making ReflectDatumReader return a Utf8, but there are a number of places in the code that expect it to return a String - e.g., a protocol test breaks and TestDataFileReflect breaks also.

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



> 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Ron Bodkin (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12918376#action_12918376 ] 

Ron Bodkin commented on AVRO-669:
---------------------------------

>From discussion on IRC, it'd be better to change the default values that ReflectData uses when reading in types to be the values that are generated by the Avro code generator (e.g., it should use GenericData.Array rather than a Java array by default). 

This will also require encoding reflection data for arrays that used to be a default, e.g., the ReflectData.createSchema method should emit a CLASS_PROP of GenericArrayType.class and reflect data reader should honor that.


> 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.


[jira] Commented: (AVRO-669) Avro Mapreduce Doesn't Work With Reflect Schemas

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/AVRO-669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12918664#action_12918664 ] 

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

I moved my patch from here to a new issue, AVRO-678, since it solves a distinct sub-problem.

> 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.