crunch-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Josh Wills <jwi...@cloudera.com>
Subject Re: crunch 0.8.2+6-cdh4.4.0
Date Thu, 06 Feb 2014 20:10:26 GMT
Hey Stephen-

Thanks for putting this together; I had an inkling of what the issue was
when you sent the stack trace and I managed to verify it using your unit
test. The org.apache.avro.mapred.Pair schema that is used in PTable isn't
defined by Crunch and doesn't allow null values. If you change your example
code to use Avros.pairs() instead of tableOf(), the test works properly,
b/c Crunch tuples do permit nulls.

I'm unsure of the best way to fix this if we want to. We could force
AvroFileTarget to convert its inputs to a Crunch tuple and not allow it to
write out an avro Pair schema, but I'd be concerned that this would break
other use cases that I'm not aware of.

J


On Thu, Feb 6, 2014 at 9:35 AM, Stephen Durfey <sjdurfey@gmail.com> wrote:

> Attached is a unit test that emits a null avro record that causes this
> stack trace to occur. The file that is being read in doesn't matter as the
> contents are ignored. I just needed to read in something to use the
> MRPipeline. Also, this test fails on both 0.8.2+6-cdh4.4.0 and
> 0.8.0-cdh4.3.0 (I haven't tried versions older than this), which leads me
> to believe that this test isn't completely representative of the complexity
> in my code base. Nevertheless, it is the exact same exception with the same
> emit behavior.
>
> Side note: In MemPipeline, when writing a PTable as a text file, there is
> an assumption that all key and value pairs are non-null. This assumption
> could be elsewhere in MemPipeline#write as well, but I haven't tried the
> other flows.
>
>
> On Wed, Jan 29, 2014 at 4:46 PM, Stephen Durfey <sjdurfey@gmail.com>wrote:
>
>> This is the full stack trace. I removed some parts of the stack trace
>> that emits the null values:
>>
>> org.apache.crunch.CrunchRuntimeException:
>> org.apache.avro.file.DataFileWriter$AppendWriteException:
>> java.lang.NullPointerException: null of <specific avro model name> in field
>> value of org.apache.avro.mapred.Pair
>>     at
>> org.apache.crunch.impl.mr.emit.MultipleOutputEmitter.emit(MultipleOutputEmitter.java:45)
>>     at org.apache.crunch.MapFn.process(MapFn.java:34)
>>     at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:99)
>>     at
>> org.apache.crunch.impl.mr.emit.IntermediateEmitter.emit(IntermediateEmitter.java:56)
>>     at org.apache.crunch.MapFn.process(MapFn.java:34)
>>     at org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:99)
>>     at
>> org.apache.crunch.impl.mr.emit.IntermediateEmitter.emit(IntermediateEmitter.java:56)
>>     <code that emits the null value>
>>     at org.apache.crunch.impl.mr.run.RTNode.cleanup(RTNode.java:118)
>>     at org.apache.crunch.impl.mr.run.RTNode.cleanup(RTNode.java:121)
>>     at org.apache.crunch.impl.mr.run.RTNode.cleanup(RTNode.java:121)
>>     at org.apache.crunch.impl.mr.run.RTNode.cleanup(RTNode.java:121)
>>     at
>> org.apache.crunch.impl.mr.run.CrunchReducer.cleanup(CrunchReducer.java:63)
>>     at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:166)
>>     at
>> org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:610)
>>     at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:444)
>>     at org.apache.hadoop.mapred.Child$4.run(Child.java:268)
>>     at java.security.AccessController.doPrivileged(Native Method)
>>     at javax.security.auth.Subject.doAs(Subject.java:396)
>>     at
>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
>>     at org.apache.hadoop.mapred.Child.main(Child.java:262)
>> Caused by: org.apache.avro.file.DataFileWriter$AppendWriteException:
>> java.lang.NullPointerException: null of <specific avro model name> in field
>> value of org.apache.avro.mapred.Pair
>>     at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:263)
>>     at
>> org.apache.crunch.types.avro.AvroOutputFormat$1.write(AvroOutputFormat.java:87)
>>     at
>> org.apache.crunch.types.avro.AvroOutputFormat$1.write(AvroOutputFormat.java:84)
>>     at org.apache.crunch.io.CrunchOutputs.write(CrunchOutputs.java:128)
>>     at
>> org.apache.crunch.impl.mr.emit.MultipleOutputEmitter.emit(MultipleOutputEmitter.java:41)
>>     ... 23 more
>> Caused by: java.lang.NullPointerException: null of <specific avro model
>> name> in field value of org.apache.avro.mapred.Pair
>>     at
>> org.apache.avro.generic.GenericDatumWriter.npe(GenericDatumWriter.java:93)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:87)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:58)
>>     at org.apache.avro.file.DataFileWriter.append(DataFileWriter.java:257)
>>     ... 27 more
>> Caused by: java.lang.NullPointerException
>>     at org.apache.avro.generic.GenericData.getField(GenericData.java:537)
>>     at org.apache.avro.generic.GenericData.getField(GenericData.java:552)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:104)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:66)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:106)
>>     at
>> org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:66)
>>     ... 29 more
>>
>>
>> On Wed, Jan 29, 2014 at 4:30 PM, Micah Whitacre <mkwhitacre@gmail.com>wrote:
>>
>>> That actually seems likely as the persistence of fixed bytes in the
>>> Buffered vs Direct encoders differs.
>>>
>>> Stephen can you include the full stack trace of the NPE which will help
>>> to verify if differences in the encoders are at fault.
>>>
>>>
>>> On Wed, Jan 29, 2014 at 4:14 PM, Josh Wills <josh.wills@gmail.com>wrote:
>>>
>>>> Maybe the binary encoder change?
>>>> On Jan 29, 2014 3:19 PM, "Durfey,Stephen" <Stephen.Durfey@cerner.com>
>>>> wrote:
>>>>
>>>>>   Sorry for the initial confusion. The exceptions that I am seeing,
>>>>> look like were caused by another exception way up in my console log (that
I
>>>>> originally missed). I think the true exception is an Avro exception.
It is
>>>>> an Avro PType, and the original NPE is coming from the
>>>>> GenericData#getField,
>>>>> when the GenericDatumWriter is serializing. The exception is for a
>>>>> null value in the org.apache.avro.mapred.Pair (I believe this Pair is
>>>>> created in PairMapFn?)  object when expecting a specific type of Avro.
I'm
>>>>> having a difficult time figuring out what changed between versions. Without
>>>>> changing
>>>>> anything in my code, and just changing Crunch versions causes these
>>>>> exceptions to be thrown.
>>>>>
>>>>>  - Stephen
>>>>>
>>>>>   From: Josh Wills <josh.wills@gmail.com>
>>>>> Reply-To: "user@crunch.apache.org" <user@crunch.apache.org>
>>>>> Date: Tuesday, January 28, 2014 at 4:08 PM
>>>>> To: "user@crunch.apache.org" <user@crunch.apache.org>
>>>>> Subject: Re: crunch 0.8.2+6-cdh4.4.0
>>>>>
>>>>>   Hey Stephen,
>>>>>
>>>>>  Slightly confused here, question inlined.
>>>>>
>>>>>
>>>>> On Tue, Jan 28, 2014 at 12:59 PM, Durfey,Stephen <
>>>>> Stephen.Durfey@cerner.com> wrote:
>>>>>
>>>>>>  This question is specifically about this version maintained by
>>>>>> Cloudera.
>>>>>>
>>>>>>  I was looking to update out Crunch version from 0.8.0-cdh4.3.0
>>>>>> to 0.8.2+6-cdh4.4.0. In the process some of our tests starting failing
from
>>>>>> NullPointerExceptions. I've discovered why these exceptions are happening,
>>>>>> but I'm having trouble tracking down the where.
>>>>>>
>>>>>>  The exceptions occur when we emit a Pair<POJO, null> that
uses an
>>>>>> Avro PType. Previously this worked just fine, and by the time the
>>>>>> CrunchOutputs started writing to a sequence file the value would
be an
>>>>>> instance of NullWritable, and it would successfully pull off the
output
>>>>>> type for serialization (in SequenceFile.BlockCompressWriter#append(k,
v)).
>>>>>> After the version change the value when it got down to write to a
sequence
>>>>>> file was 'null', rather than NullWritable.
>>>>>>
>>>>>
>>>>>  It's an AvroType that's getting written to a Sequence File? Is that
>>>>> right?
>>>>>
>>>>>
>>>>>>
>>>>>>  Any thoughts?
>>>>>>
>>>>>>  - Stephen
>>>>>> 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>

Mime
View raw message