crunch-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Micah Whitacre <mkwhita...@gmail.com>
Subject Re: Injecting alternate PType Converter implementations
Date Wed, 24 Apr 2013 23:21:38 GMT
Logged:
https://issues.apache.org/jira/browse/CRUNCH-197

Do you see any issues with adding Trevni support to Crunch?  Something
you'd rather avoid doing in lieu of waiting on Parquet?


On Wed, Apr 24, 2013 at 6:07 PM, Josh Wills <jwills@cloudera.com> wrote:

>
>
>
> On Wed, Apr 24, 2013 at 4:02 PM, Micah Whitacre <mkwhitacre@gmail.com>wrote:
>
>> > I think it's just that. It seems relatively low-risk to me (e.g., we
>> already use AvroKey in the AvroPairConverter for PTables).
>>
>> Ok sounds good.  Do you want me to log a bug for this?
>>
>
> Yes please. I'm running the small fix through regression tests now.
>
>
>>
>> > I'm also curious if you're looking at Parquet for this use case?
>>
>> Yeah was going to look at it after Trevni.  It's Avro support is not as
>> far along (looks like ~16 days).  The goal was to hopefully help get
>> support for both into Crunch eventually and we can choose whichever is
>> better for our job.
>>
>
> Fair enough.
>
>
>>
>> On Wed, Apr 24, 2013 at 5:52 PM, Josh Wills <jwills@cloudera.com> wrote:
>>
>>>
>>>
>>>
>>> On Wed, Apr 24, 2013 at 3:49 PM, Micah Whitacre <mkwhitacre@gmail.com>wrote:
>>>
>>>> Is the change simply:
>>>>
>>>>   private AvroWrapper<K> getWrapper() {
>>>>     if (wrapper == null) {
>>>>       // wrapper = new AvroWrapper<K>();
>>>>       wrapper = new AvroKey<K>();
>>>>     }
>>>>     return wrapper;
>>>>   }
>>>>
>>>> Or are there more changes I might be missing?  Doing that got me past
>>>> the ClassCastException (though still trying to get my code working).
>>>>
>>>> As I indicated I'm still just trying to prove out my code and if it
>>>> pans out we can probably wait till the 0.7.0 release (assuming the current
>>>> ~2 month release cycle).  I'll leave it to you to evaluate the risk.
>>>>
>>>
>>> I think it's just that. It seems relatively low-risk to me (e.g., we
>>> already use AvroKey in the AvroPairConverter for PTables).
>>>
>>>
>>>>
>>>> I'm guessing the injecting a converter issue will be more significant
>>>> if I try out the other Trevni format[1] where I'd need the converter to
>>>> support AvroValue instead of NullWritable.  So I'm fine with holding off
a
>>>> rushed change before a release in lieu of a more holistic solution to both
>>>> parts.
>>>>
>>>> [1] -
>>>> http://avro.apache.org/docs/current/api/java/org/apache/trevni/avro/mapreduce/AvroTrevniKeyValueOutputFormat.html
>>>>
>>>
>>> I'm also curious if you're looking at Parquet for this use case?
>>>
>>>
>>>>
>>>>
>>>>
>>>> On Wed, Apr 24, 2013 at 5:29 PM, Josh Wills <jwills@cloudera.com>wrote:
>>>>
>>>>> Hey Micah,
>>>>>
>>>>> It seems like having the AvroKeyConverter use the AvroKey as the
>>>>> return type instead of AvroWrapper is the easiest way to solve this,
since
>>>>> AvroKey is a subclass of AvroWrapper. That said, I agree, that's a thorny
>>>>> problem. We're just getting ready for the 0.6.0 release, but I'd be fine
to
>>>>> get the switch in there if that solved this problem for you.
>>>>>
>>>>> J
>>>>>
>>>>>
>>>>> On Wed, Apr 24, 2013 at 3:23 PM, Micah Whitacre <mkwhitacre@gmail.com>wrote:
>>>>>
>>>>>> As an alternative to the standard AvroInput/OutputFormat, I've been
>>>>>> playing around with how to support alternate Avro file types like
>>>>>> Trevni[1], which give benefits when we want to only retrieve a subset
of
>>>>>> the Avro object.
>>>>>>
>>>>>> Picking one of the implementations
>>>>>> (AvroTrevniKeyInputFormat/AvroTrevniKeyOutputFormat)[2], I implemented
the
>>>>>> various Source/Target/SourceTarget implementations.  When I started
trying
>>>>>> to test it out (to see if I did any of it right), I hit the issue
that the
>>>>>> AvroKeyConverter only produces AvroWrapper objects and the output
format
>>>>>> requires AvroKey.  So I get ClassCastExceptions CrunchOutputs.write(...)
>>>>>> method.
>>>>>>
>>>>>> Caused by: java.lang.ClassCastException:
>>>>>> org.apache.avro.mapred.AvroWrapper cannot be cast to
>>>>>> org.apache.avro.mapred.AvroKey
>>>>>> at
>>>>>> org.apache.trevni.avro.mapreduce.AvroTrevniKeyRecordWriter.write(AvroTrevniKeyRecordWriter.java:34)
>>>>>>  at org.apache.crunch.io.CrunchOutputs.write(CrunchOutputs.java:129)
>>>>>>
>>>>>> I was hoping that the target would be able to take any PCollection<?
>>>>>> extends AvroType> but it looks like I'd need to implement my own
PType and
>>>>>> force consumers to use that just to change the converter to produce
AvroKey
>>>>>> instead.
>>>>>>
>>>>>> Is implementing a custom PType the only way to inject an alternate
>>>>>> converter?  That seems like a high cost on the implementation side
and
>>>>>> forcing a restriction onto others in the pipeline who are generally
happy
>>>>>> with the standard AvroType and shouldn't be burdened with how the
data
>>>>>> might be stored later on in the processing.
>>>>>>
>>>>>> Thoughts?
>>>>>>
>>>>>> [1] - http://avro.apache.org/docs/current/trevni/spec.html
>>>>>> [2] -
>>>>>> http://avro.apache.org/docs/current/api/java/org/apache/trevni/avro/mapreduce/AvroTrevniKeyOutputFormat.html
>>>>>>
>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Director of Data Science
>>>>> Cloudera <http://www.cloudera.com>
>>>>> Twitter: @josh_wills <http://twitter.com/josh_wills>
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> Director of Data Science
>>> Cloudera <http://www.cloudera.com>
>>> Twitter: @josh_wills <http://twitter.com/josh_wills>
>>>
>>
>>
>
>
> --
> Director of Data Science
> Cloudera <http://www.cloudera.com>
> Twitter: @josh_wills <http://twitter.com/josh_wills>
>

Mime
View raw message