From user-return-265-apmail-crunch-user-archive=crunch.apache.org@crunch.apache.org Thu Apr 25 02:45:56 2013 Return-Path: X-Original-To: apmail-crunch-user-archive@www.apache.org Delivered-To: apmail-crunch-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 945A4F5F8 for ; Thu, 25 Apr 2013 02:45:56 +0000 (UTC) Received: (qmail 76901 invoked by uid 500); 25 Apr 2013 02:45:56 -0000 Delivered-To: apmail-crunch-user-archive@crunch.apache.org Received: (qmail 76830 invoked by uid 500); 25 Apr 2013 02:45:56 -0000 Mailing-List: contact user-help@crunch.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@crunch.apache.org Delivered-To: mailing list user@crunch.apache.org Received: (qmail 76821 invoked by uid 99); 25 Apr 2013 02:45:56 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Apr 2013 02:45:56 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: domain of jwills@cloudera.com designates 209.85.215.44 as permitted sender) Received: from [209.85.215.44] (HELO mail-la0-f44.google.com) (209.85.215.44) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 25 Apr 2013 02:45:51 +0000 Received: by mail-la0-f44.google.com with SMTP id ed20so2205871lab.17 for ; Wed, 24 Apr 2013 19:45:30 -0700 (PDT) X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=google.com; s=20120113; h=x-received:mime-version:in-reply-to:references:from:date:message-id :subject:to:content-type:x-gm-message-state; bh=6uKfNNxgaRXkKAczPnoSiOYBNXI1rLau9AWcygUNGfE=; b=MomcSJJubisEcGHPe9dabHGW3GIvr/p2us4dz5pHzrfzfqYmVR4/bjtDi+gXbt1EYE I56Clir8m/Id8Ng3BOwuyPvGWUERpfro1b2U/N4LZuQTN8oJzz+HwsiG3ZlHAEaDraIm VOS1d0DodTraASjz/xN3DpGgRatb27o3YGJo+E38bYAujqW9BtZg1wtKBgy78e46Kk/8 D+VV2scFaSSk8ynhofnAGLXIPrOoq65i2ezWra5JdZbDRmHzyEDMxxfjadha8UA1WhA2 UDo/uQ9yRF/N2lk9zOKwQzXFsAsIvzbxjvOZW/iKPvGivLg0kJP1KobCwYne+OoB4Ipk hkZw== X-Received: by 10.152.8.81 with SMTP id p17mr12176854laa.16.1366857930050; Wed, 24 Apr 2013 19:45:30 -0700 (PDT) MIME-Version: 1.0 Received: by 10.114.4.38 with HTTP; Wed, 24 Apr 2013 19:45:09 -0700 (PDT) In-Reply-To: References: From: Josh Wills Date: Wed, 24 Apr 2013 19:45:09 -0700 Message-ID: Subject: Re: Injecting alternate PType Converter implementations To: user@crunch.apache.org Content-Type: multipart/alternative; boundary=001a11c366ec6122c904db266974 X-Gm-Message-State: ALoCoQkaHwtwcxHRUZEj+H35O8Vn0MA0m1ZlKZD6A/q2w1rcMKgec/ZlLddpG4LUiaXgb3zZwwFL X-Virus-Checked: Checked by ClamAV on apache.org --001a11c366ec6122c904db266974 Content-Type: text/plain; charset=ISO-8859-1 No, I'm fine w/having Trevni support in Crunch. It should stay compatible w/whatever version of Avro we have, so no new dependency issues AFAIK. On Wed, Apr 24, 2013 at 4:21 PM, Micah Whitacre wrote: > 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 wrote: > >> >> >> >> On Wed, Apr 24, 2013 at 4:02 PM, Micah Whitacre 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 wrote: >>> >>>> >>>> >>>> >>>> On Wed, Apr 24, 2013 at 3:49 PM, Micah Whitacre wrote: >>>> >>>>> Is the change simply: >>>>> >>>>> private AvroWrapper getWrapper() { >>>>> if (wrapper == null) { >>>>> // wrapper = new AvroWrapper(); >>>>> wrapper = new AvroKey(); >>>>> } >>>>> 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 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 >>>>> > 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 >>>>>> Twitter: @josh_wills >>>>>> >>>>> >>>>> >>>> >>>> >>>> -- >>>> Director of Data Science >>>> Cloudera >>>> Twitter: @josh_wills >>>> >>> >>> >> >> >> -- >> Director of Data Science >> Cloudera >> Twitter: @josh_wills >> > > -- Director of Data Science Cloudera Twitter: @josh_wills --001a11c366ec6122c904db266974 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable
No, I'm fine w/having Trevni support in Crunch. It sho= uld stay compatible w/whatever version of Avro we have, so no new dependenc= y issues AFAIK.


On Wed, Apr 24, 2013 at 4:21 PM, Micah Whitacre <mkwhitacre@gmail.com> wrote:
Logged:

Do you see any issues with adding T= revni support to Crunch? =A0Something you'd rather avoid doing in lieu = of waiting on Parquet? =A0

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



On Wed, Apr 24, 2013 at 4:02 PM= , Micah Whitacre <mkwhitacre@gmail.com> wrote:
>=A0I think it's just tha= t. It seems relatively low-risk to me (e.g., we already use AvroKey in the = AvroPairConverter for PTables).

Ok sounds good. =A0Do you want me to log a bug for this?

Yes please. I'm running the smal= l fix through regression tests now.
=A0

>=A0I'm also curious if you're looking at Parquet for this use case?<= /span>

Yeah was going to look at it after Trevni. =A0It's Avro support is not= as far along (looks like ~16 days). =A0The goal was to hopefully help get = support for both into Crunch eventually and we can choose whichever is bett= er for our job.

Fair enough.
=A0

On Wed, Apr 24, 2= 013 at 5:52 PM, Josh Wills <jwills@cloudera.com> wrote:



On Wed, Apr 24, 2013 at 3:49 PM, Micah Whitacre <mkwhitacr= e@gmail.com> wrote:
Is the change simply:

=
=A0 private AvroWrapper<K> getWrapper() {
=A0 =A0 if (wrapp= er =3D=3D null) {
=A0 =A0 =A0 // wrapper =3D new AvroWrapper<K>();
=A0 = =A0 =A0 wrapper =3D new AvroKey<K>();
=A0 =A0 }
=A0 =A0 return wrapper;
=A0 }

Or are there more changes I might be missing? =A0Doing= that got me past the ClassCastException (though still trying to get my cod= e working).

As I indicated I'm still just trying to prove out m= y code and if it pans out we can probably wait till the 0.7.0 release (assu= ming the current ~2 month release cycle). =A0I'll leave it to you to ev= aluate the risk.=A0

I think it's just that. It= seems relatively low-risk to me (e.g., we already use AvroKey in the AvroP= airConverter for PTables).
=A0

I'm guessing the injecting a converter issue will b= e more significant if I try out the other Trevni format[1] where I'd ne= ed the converter to support AvroValue instead of NullWritable. =A0So I'= m fine with holding off a rushed change before a release in lieu of a more = holistic solution to both parts.


I'm also curious if you= 9;re looking at Parquet for this use case?
=A0



On Wed, Apr 24, 2013 at 5:29 PM, Josh Wills <jwills@cl= oudera.com> wrote:
Hey Micah,

It seems li= ke having the AvroKeyConverter use the AvroKey as the return type instead o= f 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 s= witch in there if that solved this problem for you.

J


On Wed, Apr 24, 2013 at 3:23 PM, Micah Whitacre <mkwhitac= re@gmail.com> wrote:
As an alternative to the standard AvroInp= ut/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 r= etrieve a subset of the Avro object. =A0

Picking one of the implementations (AvroTrevniKeyInputFormat= /AvroTrevniKeyOutputFormat)[2], I implemented the various Source/Target/Sou= rceTarget implementations. =A0When I started trying to test it out (to see = if I did any of it right), I hit the issue that the AvroKeyConverter only p= roduces AvroWrapper objects and the output format requires AvroKey. =A0So I= get ClassCastExceptions CrunchOutputs.write(...) method.

Caused by: java.lang.ClassCastException: org.apach= e.avro.mapred.AvroWrapper cannot be cast to org.apache.avro.mapred.AvroKey<= /div>
at org.apache.trevni= .avro.mapreduce.AvroTrevniKeyRecordWriter.write(AvroTrevniKeyRecordWriter.j= ava:34)
at org.apache.crunch.io.C= runchOutputs.write(CrunchOutputs.java:129)

I= was hoping that the target would be able to take any PCollection<? exte= nds AvroType> but it looks like I'd need to implement my own PType a= nd force consumers to use that just to change the converter to produce Avro= Key instead. =A0

Is implementing a custom PType the only way to inject a= n alternate converter? =A0That seems like a high cost on the implementation= side and forcing a restriction onto others in the pipeline who are general= ly happy with the standard AvroType and shouldn't be burdened with how = the data might be stored later on in the processing.




<= font color=3D"#888888">--
Director of Data Science
Twitter: @= josh_wills




--
Director of Data Science
Twitter: @josh_wills




--
Director of Data Science
Twitter: @josh_wills




--
=
Director of Data Science
Twitter: @josh_wills
--001a11c366ec6122c904db266974--