You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@crunch.apache.org by "Whitacre,Micah" <MI...@CERNER.COM> on 2012/12/20 19:51:58 UTC

Correct usage Thrift PTypes

We have some code that is running based off of 0.4.0-incubating.  It uses the following:

PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new HBaseSourceTarget(…));
PCollection<PersonRecord> records = raw.parallelDo(new SomeFn(), PTypes.thrifts(PersonRecord.class, raw.getTypeFamily()));

When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork because we want it to work off CDH 4.1.1 with MRv1) we now get the following exception:

java.lang.NullPointerException
        at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
        at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
        at org.apache.crunch.DoFn.setContext(DoFn.java:106)
        at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
        at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
        at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
        at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
        at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
        at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
        at org.apache.hadoop.mapred.M

This is odd because tracing through the code it essentially means that on the call to setContext(…) it is passing a null context into the setContext(…).  Other forms of our code that don't use PTypes but instead use Writable seems to work fine.  Is it invalid we are passing in Writable PTypeFamily into the PTypes methods?  The documentation is not clear about what can/can't be passed in.  Before I started creating test for this wanted to make sure we just weren't using it incorrectly.

Thanks.

CONFIDENTIALITY NOTICE This message and any included attachments are from Cerner Corporation and are intended only for the addressee. The information contained in this message is confidential and may constitute inside or non-public information under international, federal, or state securities laws. Unauthorized forwarding, printing, copying, distribution, or use of such information is strictly prohibited and may be unlawful. If you are not the addressee, please promptly delete this message and notify the sender of the delivery error by e-mail or you may call Cerner's corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024.

Re: Correct usage Thrift PTypes

Posted by Josh Wills <jw...@cloudera.com>.
Yes, to the extent that a MapFn that is involved in a derived PType (like
this one) can be used to read data into the client via a materialize()
call, where there won't be an MR context defined.


On Thu, Dec 20, 2012 at 11:56 AM, Whitacre,Micah
<MI...@cerner.com>wrote:

>  Thanks that fixed my issue, thanks for the quick turnaround.  I'm still
> probably need to trace through the code some more but is it unreasonable
> for a FN on that calling path to not expect to have the context (and
> therefore configuration set)?
>
>   On Dec 20, 2012, at 1:08 PM, Josh Wills wrote:
>
> Should be fixed now-- sorry about that. Issue is CRUNCH-137 if you need to
> re-open it.
>
>
> On Thu, Dec 20, 2012 at 11:02 AM, Josh Wills <jw...@cloudera.com> wrote:
>
>> It looks to me like we shouldn't be referencing getConfiguration() in
>> that context-- the TBase instance we create doesn't implement Configurable,
>> so the Configuration object has no effect. I'll file a bug for it and then
>> submit the one-line fix.
>>
>>
>> On Thu, Dec 20, 2012 at 10:58 AM, Josh Wills <jw...@cloudera.com> wrote:
>>
>>> Hrm, that sounds like a regression. You're saying the Thrift PTypes
>>> worked fine in 0.4.0-incubating?
>>>
>>>
>>> On Thu, Dec 20, 2012 at 10:51 AM, Whitacre,Micah <
>>> MICAH.WHITACRE@cerner.com> wrote:
>>>
>>>> We have some code that is running based off of 0.4.0-incubating.  It
>>>> uses the following:
>>>>
>>>>   PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new
>>>> HBaseSourceTarget(…));
>>>>  PCollection<PersonRecord> records = raw.parallelDo(newSomeFn(), PTypes.thrifts(PersonRecord.
>>>> class, raw.getTypeFamily()));
>>>>
>>>>  When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork
>>>> because we want it to work off CDH 4.1.1 with MRv1) we now get the
>>>> following exception:
>>>>
>>>> java.lang.NullPointerException
>>>> 	at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
>>>> 	at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
>>>> 	at org.apache.crunch.DoFn.setContext(DoFn.java:106)
>>>> 	at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
>>>> 	at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
>>>> 	at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
>>>> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
>>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
>>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>>>> 	at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
>>>> 	at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
>>>> 	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
>>>> 	at org.apache.hadoop.mapred.M
>>>>
>>>>
>>>>  This is odd because tracing through the code it essentially means
>>>> that on the call to setContext(…) it is passing a null context into the
>>>> setContext(…).  Other forms of our code that don't use PTypes but instead
>>>> use Writable seems to work fine.  Is it invalid we are passing in Writable
>>>> PTypeFamily into the PTypes methods?  The documentation is not clear about
>>>> what can/can't be passed in.  Before I started creating test for this
>>>> wanted to make sure we just weren't using it incorrectly.
>>>>
>>>>  Thanks.
>>>>  CONFIDENTIALITY NOTICE This message and any included attachments are
>>>> from Cerner Corporation and are intended only for the addressee. The
>>>> information contained in this message is confidential and may constitute
>>>> inside or non-public information under international, federal, or state
>>>> securities laws. Unauthorized forwarding, printing, copying, distribution,
>>>> or use of such information is strictly prohibited and may be unlawful. If
>>>> you are not the addressee, please promptly delete this message and notify
>>>> the sender of the delivery error by e-mail or you may call Cerner's
>>>> corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024
>>>> .
>>>>
>>>
>>>
>>>
>>>   --
>>> Director of Data Science
>>> Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
>>> Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>
>>>
>>>
>>
>>
>>  --
>> Director of Data Science
>> Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
>> Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>
>>
>>
>
>
>  --
> Director of Data Science
> Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
> Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>
>
>
>


-- 
Director of Data Science
Cloudera <http://www.cloudera.com>
Twitter: @josh_wills <http://twitter.com/josh_wills>

Re: Correct usage Thrift PTypes

Posted by "Whitacre,Micah" <MI...@CERNER.COM>.
Thanks that fixed my issue, thanks for the quick turnaround.  I'm still probably need to trace through the code some more but is it unreasonable for a FN on that calling path to not expect to have the context (and therefore configuration set)?

On Dec 20, 2012, at 1:08 PM, Josh Wills wrote:

Should be fixed now-- sorry about that. Issue is CRUNCH-137 if you need to re-open it.


On Thu, Dec 20, 2012 at 11:02 AM, Josh Wills <jw...@cloudera.com>> wrote:
It looks to me like we shouldn't be referencing getConfiguration() in that context-- the TBase instance we create doesn't implement Configurable, so the Configuration object has no effect. I'll file a bug for it and then submit the one-line fix.


On Thu, Dec 20, 2012 at 10:58 AM, Josh Wills <jw...@cloudera.com>> wrote:
Hrm, that sounds like a regression. You're saying the Thrift PTypes worked fine in 0.4.0-incubating?


On Thu, Dec 20, 2012 at 10:51 AM, Whitacre,Micah <MI...@cerner.com>> wrote:
We have some code that is running based off of 0.4.0-incubating.  It uses the following:

PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new HBaseSourceTarget(…));
PCollection<PersonRecord> records = raw.parallelDo(new SomeFn(), PTypes.thrifts(PersonRecord.class, raw.getTypeFamily()));

When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork because we want it to work off CDH 4.1.1 with MRv1) we now get the following exception:

java.lang.NullPointerException
        at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
        at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
        at org.apache.crunch.DoFn.setContext(DoFn.java:106)
        at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
        at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
        at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
        at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
        at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
        at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
        at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
        at org.apache.hadoop.mapred.M

This is odd because tracing through the code it essentially means that on the call to setContext(…) it is passing a null context into the setContext(…).  Other forms of our code that don't use PTypes but instead use Writable seems to work fine.  Is it invalid we are passing in Writable PTypeFamily into the PTypes methods?  The documentation is not clear about what can/can't be passed in.  Before I started creating test for this wanted to make sure we just weren't using it incorrectly.

Thanks.
CONFIDENTIALITY NOTICE This message and any included attachments are from Cerner Corporation and are intended only for the addressee. The information contained in this message is confidential and may constitute inside or non-public information under international, federal, or state securities laws. Unauthorized forwarding, printing, copying, distribution, or use of such information is strictly prohibited and may be unlawful. If you are not the addressee, please promptly delete this message and notify the sender of the delivery error by e-mail or you may call Cerner's corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024<tel:%28%2B1%29%20%28816%29221-1024>.



--
Director of Data Science
Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>




--
Director of Data Science
Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>




--
Director of Data Science
Cloudera<https://urldefense.proofpoint.com/v1/url?u=http://www.cloudera.com&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=69c4a3ba8590d9f34f147c88f577f3f722af47bcb676c7565b9bbbbc477f5772>
Twitter: @josh_wills<https://urldefense.proofpoint.com/v1/url?u=http://twitter.com/josh_wills&k=PmKqfXspAHNo6iYJ48Q45A%3D%3D%0A&r=MwP8zm6sgnnstbiUpAReMZvSqrZXwpejyuwyb6GLlpU%3D%0A&m=Mxzp4epdohfWSU0c8TdTK8vUW7k%2FbQtFPeDC1kYL2L4%3D%0A&s=bd5f9359f1693f0d99dbb9085a218c633473be24ab4ddf9c8de66c0575a50264>



Re: Correct usage Thrift PTypes

Posted by Josh Wills <jw...@cloudera.com>.
Should be fixed now-- sorry about that. Issue is CRUNCH-137 if you need to
re-open it.


On Thu, Dec 20, 2012 at 11:02 AM, Josh Wills <jw...@cloudera.com> wrote:

> It looks to me like we shouldn't be referencing getConfiguration() in that
> context-- the TBase instance we create doesn't implement Configurable, so
> the Configuration object has no effect. I'll file a bug for it and then
> submit the one-line fix.
>
>
> On Thu, Dec 20, 2012 at 10:58 AM, Josh Wills <jw...@cloudera.com> wrote:
>
>> Hrm, that sounds like a regression. You're saying the Thrift PTypes
>> worked fine in 0.4.0-incubating?
>>
>>
>> On Thu, Dec 20, 2012 at 10:51 AM, Whitacre,Micah <
>> MICAH.WHITACRE@cerner.com> wrote:
>>
>>>  We have some code that is running based off of 0.4.0-incubating.  It
>>> uses the following:
>>>
>>>   PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new
>>> HBaseSourceTarget(…));
>>>  PCollection<PersonRecord> records = raw.parallelDo(newSomeFn(), PTypes.thrifts(PersonRecord.
>>> class, raw.getTypeFamily()));
>>>
>>>  When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork
>>> because we want it to work off CDH 4.1.1 with MRv1) we now get the
>>> following exception:
>>>
>>> java.lang.NullPointerException
>>> 	at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
>>> 	at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
>>> 	at org.apache.crunch.DoFn.setContext(DoFn.java:106)
>>> 	at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
>>> 	at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
>>> 	at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
>>> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>>> 	at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
>>> 	at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
>>> 	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
>>> 	at org.apache.hadoop.mapred.M
>>>
>>>
>>>  This is odd because tracing through the code it essentially means that
>>> on the call to setContext(…) it is passing a null context into the
>>> setContext(…).  Other forms of our code that don't use PTypes but instead
>>> use Writable seems to work fine.  Is it invalid we are passing in Writable
>>> PTypeFamily into the PTypes methods?  The documentation is not clear about
>>> what can/can't be passed in.  Before I started creating test for this
>>> wanted to make sure we just weren't using it incorrectly.
>>>
>>>  Thanks.
>>>  CONFIDENTIALITY NOTICE This message and any included attachments are
>>> from Cerner Corporation and are intended only for the addressee. The
>>> information contained in this message is confidential and may constitute
>>> inside or non-public information under international, federal, or state
>>> securities laws. Unauthorized forwarding, printing, copying, distribution,
>>> or use of such information is strictly prohibited and may be unlawful. If
>>> you are not the addressee, please promptly delete this message and notify
>>> the sender of the delivery error by e-mail or you may call Cerner's
>>> corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024.
>>>
>>
>>
>>
>> --
>> 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>

Re: Correct usage Thrift PTypes

Posted by Josh Wills <jw...@cloudera.com>.
It looks to me like we shouldn't be referencing getConfiguration() in that
context-- the TBase instance we create doesn't implement Configurable, so
the Configuration object has no effect. I'll file a bug for it and then
submit the one-line fix.


On Thu, Dec 20, 2012 at 10:58 AM, Josh Wills <jw...@cloudera.com> wrote:

> Hrm, that sounds like a regression. You're saying the Thrift PTypes worked
> fine in 0.4.0-incubating?
>
>
> On Thu, Dec 20, 2012 at 10:51 AM, Whitacre,Micah <
> MICAH.WHITACRE@cerner.com> wrote:
>
>>  We have some code that is running based off of 0.4.0-incubating.  It
>> uses the following:
>>
>>   PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new
>> HBaseSourceTarget(…));
>>  PCollection<PersonRecord> records = raw.parallelDo(newSomeFn(), PTypes.thrifts(PersonRecord.
>> class, raw.getTypeFamily()));
>>
>>  When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork because
>> we want it to work off CDH 4.1.1 with MRv1) we now get the following
>> exception:
>>
>> java.lang.NullPointerException
>> 	at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
>> 	at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
>> 	at org.apache.crunch.DoFn.setContext(DoFn.java:106)
>> 	at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
>> 	at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
>> 	at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
>> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
>> 	at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
>> 	at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
>> 	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
>> 	at org.apache.hadoop.mapred.M
>>
>>
>>  This is odd because tracing through the code it essentially means that
>> on the call to setContext(…) it is passing a null context into the
>> setContext(…).  Other forms of our code that don't use PTypes but instead
>> use Writable seems to work fine.  Is it invalid we are passing in Writable
>> PTypeFamily into the PTypes methods?  The documentation is not clear about
>> what can/can't be passed in.  Before I started creating test for this
>> wanted to make sure we just weren't using it incorrectly.
>>
>>  Thanks.
>>  CONFIDENTIALITY NOTICE This message and any included attachments are
>> from Cerner Corporation and are intended only for the addressee. The
>> information contained in this message is confidential and may constitute
>> inside or non-public information under international, federal, or state
>> securities laws. Unauthorized forwarding, printing, copying, distribution,
>> or use of such information is strictly prohibited and may be unlawful. If
>> you are not the addressee, please promptly delete this message and notify
>> the sender of the delivery error by e-mail or you may call Cerner's
>> corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024.
>>
>
>
>
> --
> 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>

Re: Correct usage Thrift PTypes

Posted by Josh Wills <jw...@cloudera.com>.
Hrm, that sounds like a regression. You're saying the Thrift PTypes worked
fine in 0.4.0-incubating?


On Thu, Dec 20, 2012 at 10:51 AM, Whitacre,Micah
<MI...@cerner.com>wrote:

>  We have some code that is running based off of 0.4.0-incubating.  It uses
> the following:
>
>   PTable <ImmutableBytesWritable, Result> raw = pipeline.read(new
> HBaseSourceTarget(…));
>  PCollection<PersonRecord> records = raw.parallelDo(newSomeFn(), PTypes.thrifts(PersonRecord.
> class, raw.getTypeFamily()));
>
>  When upgrading to 0.5.0-incubating-SNAPSHOT (technically a fork because
> we want it to work off CDH 4.1.1 with MRv1) we now get the following
> exception:
>
> java.lang.NullPointerException
> 	at org.apache.crunch.DoFn.getConfiguration(DoFn.java:128)
> 	at org.apache.crunch.types.PTypes$ThriftInputMapFn.initialize(PTypes.java:167)
> 	at org.apache.crunch.DoFn.setContext(DoFn.java:106)
> 	at org.apache.crunch.fn.CompositeMapFn.initialize(CompositeMapFn.java:37)
> 	at org.apache.crunch.types.writable.WritableType.initialize(WritableType.java:109)
> 	at org.apache.crunch.types.writable.WritableTableType.initialize(WritableTableType.java:107)
> 	at org.apache.crunch.impl.mr.emit.IntermediateEmitter.<init>(IntermediateEmitter.java:47)
> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:78)
> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
> 	at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:68)
> 	at org.apache.crunch.impl.mr.run.CrunchTaskContext.getNodes(CrunchTaskContext.java:57)
> 	at org.apache.crunch.impl.mr.run.CrunchMapper.setup(CrunchMapper.java:41)
> 	at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:138)
> 	at org.apache.hadoop.mapred.M
>
>
>  This is odd because tracing through the code it essentially means that
> on the call to setContext(…) it is passing a null context into the
> setContext(…).  Other forms of our code that don't use PTypes but instead
> use Writable seems to work fine.  Is it invalid we are passing in Writable
> PTypeFamily into the PTypes methods?  The documentation is not clear about
> what can/can't be passed in.  Before I started creating test for this
> wanted to make sure we just weren't using it incorrectly.
>
>  Thanks.
>  CONFIDENTIALITY NOTICE This message and any included attachments are
> from Cerner Corporation and are intended only for the addressee. The
> information contained in this message is confidential and may constitute
> inside or non-public information under international, federal, or state
> securities laws. Unauthorized forwarding, printing, copying, distribution,
> or use of such information is strictly prohibited and may be unlawful. If
> you are not the addressee, please promptly delete this message and notify
> the sender of the delivery error by e-mail or you may call Cerner's
> corporate offices in Kansas City, Missouri, U.S.A at (+1) (816)221-1024.
>



-- 
Director of Data Science
Cloudera <http://www.cloudera.com>
Twitter: @josh_wills <http://twitter.com/josh_wills>