Okay...I think I need some more context around what is preceding this function. What is everything that happens between the outer join (using a JoinStrategy?) and this MapFn call?
On Mon, Jun 1, 2015 at 4:47 PM, David Ortiz <[email protected]> wrote: > I can’t say for sure it was an empty one that threw the exception, but > that is processing the output of an outer join, so they definitely exist. > > > > *From:* Josh Wills [mailto:[email protected]] > *Sent:* Monday, June 01, 2015 4:42 PM > > *To:* [email protected] > *Subject:* Re: Iteratable Bug? > > > > I can't replicate it easily in master-- is the iterable in question empty > by any chance? > > > > On Mon, Jun 1, 2015 at 2:34 PM, David Ortiz <[email protected]> > wrote: > > This is 0.11.0-cdh5.3.2 > > > > *From:* Josh Wills [mailto:[email protected]] > *Sent:* Monday, June 01, 2015 2:33 PM > > > *To:* [email protected] > *Subject:* Re: Iteratable Bug? > > > > Yeah, that's odd. This is 0.12? Let me see if I can reproduce it. > > > > J > > > > On Mon, Jun 1, 2015 at 2:31 PM, David Ortiz <[email protected]> > wrote: > > Hello Josh, > > > > Sorry, it is the next() that is throwing the exception. > > > > Thanks, > > Dave > > > > *From:* Josh Wills [mailto:[email protected]] > *Sent:* Monday, June 01, 2015 2:30 PM > *To:* [email protected] > *Subject:* Re: Iteratable Bug? > > > > Hey David, > > > > It seems like it. Which line in the function is throwing the exception? Is > it the hasNext(), or the next()? > > > > J > > > > On Mon, Jun 1, 2015 at 2:07 PM, David Ortiz <[email protected]> > wrote: > > Hello everyone, > > > > I noticed the following does not work in my pipeline: > > > > @Override > *public *MergedAttrMapping map(Pair<AttrMapping, Iterable<Integer>> > attrMappingIterablePair) { > MergedAttrMapping out = *mapper*.map(attrMappingIterablePair.first()); > StringBuilder ids = *new *StringBuilder(); > > Iterator<Integer> it = attrMappingIterablePair.second().iterator(); > while (it.hasNext()) { > > Integer I = it.next(); > > *if *(i != *null *&& i != 0) { > ids.append(i); > ids.append(*'|'*); > } > } > > *if *(ids.length() > 0) { > ids.deleteCharAt(ids.length() -1); > } > > out.setIds(ids.toString()); > > *return *out; > } > > > > Causing the following exception: > > > > Error: java.util.NoSuchElementException: iterate past last value at > org.apache.hadoop.mapreduce.task.ReduceContextImpl$ValueIterator.next(ReduceContextImpl.java:235) > at > org.apache.crunch.types.avro.AvroPairConverter$AvroWrappedIterable$1.next(AvroPairConverter.java:103) > at > org.apache.crunch.types.PGroupedTableType$HoldLastIterator.next(PGroupedTableType.java:84) > at com.videologygroup.crunch.FteWarehouse$1.map(FteWarehouse.java:268) at > com.videologygroup.crunch.FteWarehouse$1.map(FteWarehouse.java:257) at > org.apache.crunch.MapFn.process(MapFn.java:34) at > org.apache.crunch.impl.mr.run.RTNode.process(RTNode.java:98) 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:98) at > org.apache.crunch.impl.mr.run.RTNode.processIterable(RTNode.java:113) at > org.apache.crunch.impl.mr.run.CrunchReducer.reduce(CrunchReducer.java:57) > at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:171) at > org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:627) at > org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:389) at > org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168) at > java.security.AccessController.doPrivileged(Native Method) at > javax.security.auth.Subject.doAs(Subject.java:415) at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1642) > at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163) > > > > but when I change it to this (change highlighted in yellow): > > > > @Override > *public *MergedAttrMapping map(Pair<AttrMapping, Iterable<Integer>> > attrMappingIterablePair) { > MergedAttrMapping out = *mapper*.map(attrMappingIterablePair.first()); > StringBuilder dmpAttrs = *new *StringBuilder(); > > *for *(Integer i : attrMappingIterablePair.second()) { > *if *(i != *null *&& i != 0) { > ids.append(i); > ids.append(*'|'*); > } > } > > *if *(ids.length() > 0) { > ids.deleteCharAt(ids.length() -1); > } > > out.setIds(ids.toString()); > > *return *out; > } > > > > It does. > > > > Is this a bug? > > > > Thanks, > > Dave Ortiz > > *This email is intended only for the use of the individual(s) to whom it > is addressed. If you have received this communication in error, please > immediately notify the sender and delete the original email.* > > > > > > -- > > Director of Data Science > > Cloudera <http://www.cloudera.com> > > Twitter: @josh_wills <http://twitter.com/josh_wills> > > *This email is intended only for the use of the individual(s) to whom it > is addressed. If you have received this communication in error, please > immediately notify the sender and delete the original email.* > > > > > > -- > > Director of Data Science > > Cloudera <http://www.cloudera.com> > > Twitter: @josh_wills <http://twitter.com/josh_wills> > > *This email is intended only for the use of the individual(s) to whom it > is addressed. If you have received this communication in error, please > immediately notify the sender and delete the original email.* > > > > > > -- > > Director of Data Science > > Cloudera <http://www.cloudera.com> > > Twitter: @josh_wills <http://twitter.com/josh_wills> > *This email is intended only for the use of the individual(s) to whom > it is addressed. If you have received this communication in error, please > immediately notify the sender and delete the original email.* > -- Director of Data Science Cloudera <http://www.cloudera.com> Twitter: @josh_wills <http://twitter.com/josh_wills>
