Return-Path: Delivered-To: apmail-avro-user-archive@www.apache.org Received: (qmail 21592 invoked from network); 10 Mar 2011 22:29:13 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 10 Mar 2011 22:29:13 -0000 Received: (qmail 16743 invoked by uid 500); 10 Mar 2011 22:29:13 -0000 Delivered-To: apmail-avro-user-archive@avro.apache.org Received: (qmail 16707 invoked by uid 500); 10 Mar 2011 22:29:13 -0000 Mailing-List: contact user-help@avro.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@avro.apache.org Delivered-To: mailing list user@avro.apache.org Received: (qmail 16699 invoked by uid 99); 10 Mar 2011 22:29:13 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Mar 2011 22:29:13 +0000 X-ASF-Spam-Status: No, hits=2.2 required=5.0 tests=FREEMAIL_FROM,HTML_MESSAGE,RCVD_IN_DNSWL_NONE,RFC_ABUSE_POST,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of eychih@hotmail.com designates 65.55.90.172 as permitted sender) Received: from [65.55.90.172] (HELO snt0-omc3-s33.snt0.hotmail.com) (65.55.90.172) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Mar 2011 22:29:03 +0000 Received: from SNT113-W39 ([65.55.90.137]) by snt0-omc3-s33.snt0.hotmail.com with Microsoft SMTPSVC(6.0.3790.4675); Thu, 10 Mar 2011 14:28:41 -0800 Message-ID: Content-Type: multipart/alternative; boundary="_885d0a80-99ff-4cdb-b101-272536beb62a_" X-Originating-IP: [173.227.61.3] From: ey-chih chow To: Subject: RE: is this a bug? Date: Thu, 10 Mar 2011 14:28:41 -0800 Importance: Normal In-Reply-To: References: ,, MIME-Version: 1.0 X-OriginalArrivalTime: 10 Mar 2011 22:28:41.0808 (UTC) FILETIME=[8273BD00:01CBDF72] X-Virus-Checked: Checked by ClamAV on apache.org --_885d0a80-99ff-4cdb-b101-272536beb62a_ Content-Type: text/plain; charset="iso-8859-1" Content-Transfer-Encoding: quoted-printable I changed the Games__ field of the DeviceRow to union {null=2C array} Games__=3B=20 the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow From: eychih@hotmail.com To: user@avro.apache.org Subject: RE: is this a bug? Date: Thu=2C 10 Mar 2011 11:33:13 -0800 Thanks. I tried to migrate to 1.5.0 from 1.4.0. I came up with some error= messages that are never shown up for 1.4.0. Could you tell me what we sho= uld change? Our avdl record=2C DeviceRow=2C has a field defined as follows= : union {array=2C null} Games__=3B The error messages are as follows: 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_2011031011= 3041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbas= e.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.Dev= iceRow at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWriter.= java:104) at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.= java:57) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.seria= lize(AvroSerialization.java:131) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.seria= lize(AvroSerialization.java:114) at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.jav= a:900) at org.apache.hadoop.mapred.MapTask$OldOutputCollector.collect(MapTask.= java:466) at org.apache.avro.mapred.HadoopMapper$MapCollector.collect(HadoopMappe= r.java:69) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivity= GatheringMapper.java:91) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivity= GatheringMapper.java:1) at org.apache.avro.mapred.HadoopMapper.map(HadoopMapper.java:80) at org.apache.avro.mapred.HadoopMapper.map(HadoopMapper.java:34) at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50) at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:307) at org.apache.hadoop.mapred.Child.main(Child.java:170) Caused by: java.lang.NullPointerException: in union null of union in field = Games__ of com.ngmoco.hbase.DeviceRow at org.apache.avro.generic.GenericDatumWriter.npe(GenericDatumWriter.ja= va:92) at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.= java:86) at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWriter.= java:102) ... 14 more From: scott@richrelevance.com To: user@avro.apache.org Date: Tue=2C 8 Mar 2011 15:06:20 -0800 Subject: Re: is this a bug? I haven't completely gone through your messages to understand your problem = completely. However=2C there were a couple fixes in 1.5.0 that could be re= lated. What happens if you use the 1.5.0 release candidate? Staged maven repository for release candidate: https://repository.apache.or= g/content/repositories/orgapacheavro-001/release candidate: http://people.a= pache.org/~cutting/avro-1.5.0-rc3/ Note there are some API changes that may affect you a little=2C see CHANGES= .txt -Scott On 3/8/11 2:35 PM=2C "ey-chih chow" wrote: Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 i= n all of our code. Some of the jobs show weird behavior. We want to know = if this is fixable. Otherwise=2C we have to take out all the avro APIs and= use the conventional MR APIs instead. Ey-Chih Chow From: eychih@hotmail.com To: user@avro.apache.org Subject: RE: is this a bug? Date: Fri=2C 4 Mar 2011 16:57:02 -0800 I did some more investigation. I found weird behavior in the readString() = method of BinaryDecoder.java in Avro source code if we have the statement r= ecord.put("rowkey"=2C key) in the reduce() method. Does this mean that the= re is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow=20 From: eychih@hotmail.com To: user@avro.apache.org Subject: RE: is this a bug? Date: Fri=2C 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respe= ctively to with and without the statement 'record.put("rowkey"=2C key)' men= tioned in the previous messages. From the last line=2C logged at the entry= of the reduce() method=2C of each of these two logs you can see the differ= ence. I.e. for the first segment=2C the log is 'working on 000000020000000= 0000000000000000000002 whose rowKey is 000000030000000000000000000000000000= 3' for the second segment=2C the log is 'working on 00000002000000000000000= 00000000000002 whose rowKey is 0000000200000000000000000000000000002'=2C w= here the second log is what we expected=2C corresponding to the correct key= values pair passed to the reduce() method. Note that these two fragments = of logs are generated by adding some additional log statements to Hadoop an= d Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D log fragment with the statement 'record.put("rowkey"=2C key) 2011-03-03 18:00:00=2C180 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug isSkipping():false 2011-03-03 18:00:00=2C190 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 18:00:00=2C198 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 18:00:00=2C199 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug done with set values 2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000000000000000000000000000000000 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00=2C199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00=2C199 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 18:00:00=2C208 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 18:00:00=2C208 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000000000000000000000000000000000= 0"=2C "tableName": null=2C "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.= 5"=2C "columnValue": {"bytes": "ame": "hwty"=2C "columnValue": "stringvalue= "}=2C {"columnName": "loc"=2C "columnValue": "stringvalue"}=2C {"columnName= ": "osrev"=2C "columnValue": "stringvalue"}=2C {"columnName": "tz"=2C "colu= mnValue": "stringvalue"}]=2C "PlayerState__": [{"columnName": "0_TESTFAM_TE= STSKU_1.0=3DGC=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.= 99=2C"hat":"red"}"=2C "timestamp": 123456789}=2C {"columnName": "0_TESTFAM_= TESTSKU_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":= 1.99=2C"hat":"red"}"=2C "timestamp": 123456799}]=2C "ClientSessions__": nul= l=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 18:00:00=2C208 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000000000000000000000000000000000"=2C "tableNa= me": null=2C"Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5"=2C "columnVa= lue": {"bytes": "ame": "hwty"=2C "columnValue": "stringvalue"}=2C {"columnN= ame": "loc"=2C "columnValue": "stringvalue"}=2C {"columnName": "osrev"=2C "= columnValue": "stringvalue"}=2C {"columnName": "tz"=2C "columnValue": "stri= ngvalue"}]=2C "PlayerState__": [{"columnName":"0_TESTFAM_TESTSKU_1.0=3DGC= =3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"re= d"}"=2C "timestamp": 123456789}=2C {"columnName":"0_TESTFAM_TESTSKU_1.0=3DG= S=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"r= ed"}"=2C "timestamp": 123456799}]=2C "ClientSessions__": null=2C "ServerSes= sions__": null=2C "Monetization__": null} 2011-03-03 18:00:00=2C208 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 18:00:00=2C208 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 18:00:00=2C208 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 18:00:00=2C209 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000100000000000000000000000000001 2011-03-03 18:00:00=2C209 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 18:00:00=2C209 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000100000000000000000000000000001 2011-03-03 18:00:00=2C210 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000000000000000000000000000000000 whose rowKey i= s 0000000000000000000000000000000000000 2011-03-03 18:00:00=2C215 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug call nextKey() 2011-03-03 18:00:00=2C215 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000100000000000000000000000000001 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00=2C215 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000100000000000000000000000000001 2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 18:00:00=2C216 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000010000000000000000000000000000= 1"=2C "tableName":null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [{= "columnName": "0_TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValu= e": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C"timestamp": 123456789}]= =2C "ClientSessions__": null=2C "ServerSessions__": null=2C "Monetization__= ": null} 2011-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000100000000000000000000000000001"=2C "tableNa= me": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "= 0_TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 18:00:00=2C216 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000200000000000000000000000000002 2011-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000200000000000000000000000000002 2011-03-03 18:00:00=2C216 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000100000000000000000000000000001 whose rowKey i= s 0000000200000000000000000000000000002 2011-03-03 18:00:00=2C217 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug call nextKey() 2011-03-03 18:00:00=2C217 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000200000000000000000000000000002 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00=2C217 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000200000000000000000000000000002 2011-03-03 18:00:00=2C217 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 18:00:00=2C217 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 18:00:00=2C218 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000020000000000000000000000000000= 2"=2C "tableName": null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [= {"columnName": "0_TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnVal= ue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}= ]=2C "ClientSessions__": null=2C "ServerSessions__": null=2C "Monetization_= _": null} 2011-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000200000000000000000000000000002"=2C "tableNa= me": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "= 0_TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 18:00:00=2C218 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 18:00:00=2C218 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 18:00:00=2C218 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000300000000000000000000000000003 2011-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000300000000000000000000000000003 2011-03-03 18:00:00=2C218 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000200000000000000000000000000002 whose rowKey i= s 0000000300000000000000000000000000003 =20 =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D log fragment without the statement 'record.put("rowkey"=2C key) 2011-03-03 21:02:05=2C077 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug isSkipping():false 2011-03-03 21:02:05=2C092 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 21:02:05=2C102 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 21:02:05=2C102 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000000000000000000000000000000000 2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug done with set values 2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000000000000000000000000000000000 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 21:02:05=2C103 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000000000000000000000000000000000 2011-03-03 21:02:05=2C103 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 21:02:05=2C113 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 21:02:05=2C114 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000000000000000000000000000000000= 0"=2C "tableName": null=2C "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.= 5"=2C "columnValue": {"bytes": "ame": "hwty"=2C "columnValue": "stringvalue= "}=2C {"columnName": "loc"=2C "columnValue": "stringvalue"}=2C {"columnName= ": "osrev"=2C "columnValue": "stringvalue"}=2C {"columnName": "tz"=2C "colu= mnValue": "stringvalue"}]=2C "PlayerState__": [{"columnName": "0_TESTFAM_TE= STSKU_1.0=3DGC=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.= 99=2C"hat":"red"}"=2C "timestamp": 123456789}=2C {"columnName": "0_TESTFAM_= TESTSKU_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":= 1.99=2C"hat":"red"}"=2C "timestamp": 123456799}]=2C "ClientSessions__": nul= l=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000000000000000000000000000000000"=2C "tableNa= me": null=2C "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5"=2C "columnV= alue": {"bytes": "ame": "hwty"=2C "columnValue": "stringvalue"}=2C {"column= Name": "loc"=2C "columnValue": "stringvalue"}=2C {"columnName": "osrev"=2C = "columnValue": "stringvalue"}=2C {"columnName": "tz"=2C "columnValue": "str= ingvalue"}]=2C "PlayerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=3DGC= =3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"re= d"}"=2C "timestamp": 123456789}=2C {"columnName": "0_TESTFAM_TESTSKU_1.0=3D= GS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"= red"}"=2C "timestamp": 123456799}]=2C "ClientSessions__": null=2C "ServerSe= ssions__": null=2C "Monetization__": null} 2011-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 21:02:05=2C114 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 21:02:05=2C114 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 21:02:05=2C114 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000100000000000000000000000000001 2011-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000100000000000000000000000000001 2011-03-03 21:02:05=2C115 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000000000000000000000000000000000 whose rowKey i= s 0000000000000000000000000000000000000 2011-03-03 21:02:05=2C121 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug call nextKey() 2011-03-03 21:02:05=2C121 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000100000000000000000000000000001 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 21:02:05=2C121 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000100000000000000000000000000001 2011-03-03 21:02:05=2C121 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 21:02:05=2C121 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 21:02:05=2C121 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000010000000000000000000000000000= 1"=2C "tableName": null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [= {"columnName": "0_TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnVal= ue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}= ]=2C "ClientSessions__": null=2C "ServerSessions__": null=2C "Monetization_= _": null} 2011-03-03 21:02:05=2C121 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000100000000000000000000000000001"=2C "tableNa= me": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "= 0_TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 21:02:05=2C122 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 21:02:05=2C122 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 21:02:05=2C122 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 21:02:05=2C122 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000200000000000000000000000000002 2011-03-03 21:02:05=2C122 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 21:02:05=2C122 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000200000000000000000000000000002 2011-03-03 21:02:05=2C122 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000100000000000000000000000000001 whose rowKey i= s 0000000100000000000000000000000000001 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug call nextKey() 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.ReduceTask: trace b= ug key is 0000000200000000000000000000000000002 values is org.apache.hadoop= .mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 21:02:05=2C123 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: work on key 0000000200000000000000000000000000002 2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@2= 6e9f9 2011-03-03 21:02:05=2C123 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected RECORD 2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum {"rowKey": "000000020000000000000000000000000000= 2"=2C "tableName": null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [= {"columnName": "0_TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnVal= ue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}= ]=2C "ClientSessions__": null=2C "ServerSessions__": null=2C "Monetization_= _": null} 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 value is {"rowKey": "0000000200000000000000000000000000002"=2C "tableNa= me": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "= 0_TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null} 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.Merger: trace bug a= djust priority queue 2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1= a001ff 2011-03-03 21:02:05=2C123 INFO org.apache.avro.generic.GenericDatumReader: = trace bug type of expected STRING 2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSerialization: tr= ace bug deserialized datum 0000000300000000000000000000000000003 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDes= erializer@1abcc03 2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.TaskRunner: trace b= ug1 key is 0000000300000000000000000000000000003 2011-03-03 21:02:05=2C124 INFO com.ngmoco.ngpipes.sourcing.NgActivityGather= ingReducer: working on 0000000200000000000000000000000000002 whose rowKey i= s 0000000200000000000000000000000000002 From: eychih@hotmail.com To: user@avro.apache.org Subject: RE: is this a bug? Date: Wed=2C 2 Mar 2011 16:12:20 -0800 Sorry I found that my previous message in the archive become all in black. = Let me re-explain the problem. The following piece of code for AvroReduce= r causes problem: public void reduce(Utf8 key=2C Iterable values=2C= AvroCollector collector=2C Reporter reporter) throws IOExce= ption { GenericRecord record =3D null=3B = for (GenericRecord value : values) { = -- code omitted here -- record =3D = value=3B record.put("rowkey"=2C key)=3B <= =3D=3D=3D this statement causes problem col= lector.collect(record)=3B } } As explained in my previous message=2C if I remove the statement record.put= ("rowkey"=2C key)=2C the code works fine=2C in that the key values pairs pa= ssed to the routine reduce() are correct. But if you add this statement=2C= the key values pairs passed to the routine reduce() are out of order=2C so= mething like (key1=2C values1)=2C (key2=2C values3) rather than (key2=2C va= lues2). Some details are explained in my previous message. Is this probl= em relating to Hadoop binary iterators or Avro deserialization code? Thank= s. Ey-Chih Chow From: eychih@hotmail.com To: user@avro.apache.org Subject: is this a bug? Date: Wed=2C 2 Mar 2011 13:05:55 -0800 Hi=2C I am working on an Avro MR job and encountering an issue with AvroReducer. The corresponding reduce() routine = is implemented in the following way: public void reduce(Utf8 key=2C Iterable values=2C AvroCollec= tor collector=2C Reporter reporter) throws IOException { . . = . GenericRecord record =3D null=3B for (GenericRecord value : values) { = . . = . record =3D value=3B record.put("rowkey"=2C key)=3B = . . = . collector.collect(record)=3B = }}=20 If I comment out the statement in red in the above code=2C the reduce funct= ion gets called properly with CORRECT key values pairs passed to reduce(). = However=2C if I add the statement in red to the routine=2C the reduce func= tion is called with WRONG key values pairs=2C in the sense that key2 paired= with values3=2C instead of values2=2C when passed to the reduce() routine.= I traced this problem by including Hadoop source code=2C such as ReduceTa= sk.java=2C Task.java=2C and Avro source code=2C such as HadoopReducer.java= =2C HadoopReducerBase.java=2C and all the serialization code. The problem = showed up on the second call of the reduce()=2C but I can not locate the ex= act place that cause the problem. My intuition is that this is incurred in= either the hadoop iterators after merge sort or Avro deserialization. Is = there anybody can help me on this? Thanks. Ey-Chih Chow = --_885d0a80-99ff-4cdb-b101-272536beb62a_ Content-Type: text/html; charset="iso-8859-1" Content-Transfer-Encoding: quoted-printable I changed the Games__ field of the DeviceRow to

union {n= ull=2C array<=3BDynamicColumn4Games>=3B} Games__=3B =3B
<= br>
the system seemed no longer complaining.  =3BIs this a ri= ght fix?  =3BThanks.

Ey-Chih Chow


From: eychih@hotmail.com
To: user@avro.apache.org
= Subject: RE: is this a bug?
Date: Thu=2C 10 Mar 2011 11:33:13 -0800
<= br> Thanks. =3B I tried to migrate to 1.5.0 from 1.4.0. =3B I came up w= ith some error messages that are never shown up for 1.4.0. =3B Could yo= u tell me what we should change? =3B Our avdl record=2C DeviceRow=2C ha= s a field defined as follows:

union {array<=3BDynamicColumn4Games&= gt=3B=2C null} Games__=3B

The error messages are as follows:

= 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_2011031011= 3041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbas= e.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.Dev= iceRow
 =3B =3B =3B at org.apache.avro.reflect.ReflectDatumW= riter.write(ReflectDatumWriter.java:104)
 =3B =3B =3B at org= .apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:57) =3B =3B =3B at org.apache.avro.mapred.AvroSerialization$Avro= WrapperSerializer.serialize(AvroSerialization.java:131)
 =3B =3B=  =3B at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.= serialize(AvroSerialization.java:114)
 =3B =3B =3B at org.ap= ache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:900)
&nb= sp=3B =3B =3B at org.apache.hadoop.mapred.MapTask$OldOutputCollecto= r.collect(MapTask.java:466)
 =3B =3B =3B at org.apache.avro.= mapred.HadoopMapper$MapCollector.collect(HadoopMapper.java:69)
 =3B&= nbsp=3B =3B at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.ma= p(NgActivityGatheringMapper.java:91)
 =3B =3B =3B at com.ngm= oco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMappe= r.java:1)
 =3B =3B =3B at org.apache.avro.mapred.HadoopMappe= r.map(HadoopMapper.java:80)
 =3B =3B =3B at org.apache.avro.= mapred.HadoopMapper.map(HadoopMapper.java:34)
 =3B =3B =3B a= t org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50)
 =3B&nbs= p=3B =3B at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:= 358)
 =3B =3B =3B at org.apache.hadoop.mapred.MapTask.run(Ma= pTask.java:307)
 =3B =3B =3B at org.apache.hadoop.mapred.Chi= ld.main(Child.java:170)
Caused by: java.lang.NullPointerException: in un= ion null of union in field Games__ of com.ngmoco.hbase.DeviceRow
 = =3B =3B =3B at org.apache.avro.generic.GenericDatumWriter.npe(Gener= icDatumWriter.java:92)
 =3B =3B =3B at org.apache.avro.gener= ic.GenericDatumWriter.write(GenericDatumWriter.java:86)
 =3B =3B=  =3B at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWr= iter.java:102)
 =3B =3B =3B ... 14 more


From: scott@richrelevance.com
To: user@avro.apache.org
= Date: Tue=2C 8 Mar 2011 15:06:20 -0800
Subject: Re: is this a bug?

I haven't completely gone through your messages to unde= rstand your problem completely.  =3BHowever=2C there were a couple fixe= s in 1.5.0 that could be related.

What happens if = you use the 1.5.0 release candidate?

Staged maven = repository for release candidate: =3Bhttps://r= epository.apache.org/content/repositories/orgapacheavro-001/

Note there are some API changes that may= affect you a little=2C see CHANGES.txt

-Scott

On 3/8/11 2= :35 PM=2C "ey-chih chow" <=3Beychih= @hotmail.com>=3B wrote:

Can anybody tell me if this this a bug?  =3BWe use avro map/reduce API = v 1.4 in all of our code.  =3BSome of the jobs show weird behavior. &nb= sp=3BWe want to know if this is fixable.  =3BOtherwise=2C we have to ta= ke out all the avro APIs and use the conventional MR APIs instead.

=
Ey-Chih Chow


From: eychih@hotmail.com
To: user@avro.apache.org
Subject: RE: is this a bu= g?
Date: Fri=2C 4 Mar 2011 16:57:02 -0800

I did some more investigation.  =3BI found weird behavior in the readSt= ring() method of BinaryDecoder.java in Avro source code if we have the stat= ement record.put("rowkey"=2C key) in the reduce() method.  =3BDoes this= mean that there is a bug in BinaryDecoder.java ?  =3BThanks.

<= /div>
Ey-Chih Chow =3B


From: eychih@hotmail.com
To: user@avro.apache.org
Subject: RE: is th= is a bug?
Date: Fri=2C 4 Mar 2011 00:48:55 -0800

What follows are fragments of trace logs of our MR jobs corresponding respe= ctively to with and without the statement 'record.put("rowkey"=2C key)' men= tioned in the previous messages. =3B From the last line=2C logged at th= e entry of the reduce() method=2C of each of these two logs you can see the= difference. =3B I.e. for the first segment=2C the log is 'working on 0= 000000200000000000000000000000000002 whose rowKey is 0000000300000000000000= 000000000000003' for the second segment=2C the log is 'working on 000000020= 0000000000000000000000000002 whose rowKey is 000000020000000000000000000000= 0000002'=2C =3B where the second log is what we expected=2C correspondi= ng to the correct key values pair passed to the reduce() method. =3B No= te that these two fragments of logs are generated by adding some additional= log statements to Hadoop and Avro source code.

Can anybody help to = see if this is a bug in Avro or Hadoop code?

=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D

log fragment with the statement 'record.put("rowkey"=2C= key)

2011-03-03 18:00:00=2C180 INFO org.apache.hadoop.mapred.Re= duceTask: trace bug isSkipping():false
2011-03-03 18:00:00=2C190 INFO or= g.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.= apache.avro.specific.SpecificDatumReader@1a001ff
2011-03-03 18:00:00=2C1= 98 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expec= ted STRING
2011-03-03 18:00:00=2C199 INFO org.apache.avro.mapred.AvroSer= ialization: trace bug deserialized datum 0000000000000000000000000000000000= 000
2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.TaskRunner: = trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWra= pperDeserializer@1abcc03
2011-03-03 18:00:00=2C199 INFO org.apache.hadoo= p.mapred.TaskRunner: trace bug1 key is 000000000000000000000000000000000000= 0
2011-03-03 18:00:00=2C199 INFO org.apache.hadoop.mapred.ReduceTask: tr= ace bug done with set values
2011-03-03 18:00:00=2C199 INFO org.apache.h= adoop.mapred.ReduceTask: trace bug key is 000000000000000000000000000000000= 0000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1de= eb40
2011-03-03 18:00:00=2C199 INFO com.ngmoco.ngpipes.sourcing.NgActivi= tyGatheringReducer: work on key 0000000000000000000000000000000000000
20= 11-03-03 18:00:00=2C199 INFO org.apache.avro.mapred.AvroSerialization: trac= e bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e= 9f9
2011-03-03 18:00:00=2C208 INFO org.apache.avro.generic.GenericDatumR= eader: trace bug type of expected RECORD
2011-03-03 18:00:00=2C208 INFO = org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"ro= wKey": "0000000000000000000000000000000000000"=2C "tableName": null=2C "Gam= es__": [{"columnName": "0_TESTFAM_TESTSKU_1.5"=2C "columnValue": {"bytes": = "ame": "hwty"=2C "columnValue": "stringvalue"}=2C {"columnName": "loc"=2C "= columnValue": "stringvalue"}=2C {"columnName": "osrev"=2C "columnValue": "s= tringvalue"}=2C {"columnName": "tz"=2C "columnValue": "stringvalue"}]=2C "P= layerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=3DGC=3D2010:01:01:07"= =2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestam= p": 123456789}=2C {"columnName": "0_TESTFAM_TESTSKU_1.0=3DGS=3D2010:01:01:0= 7"=2C "columnValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timest= amp": 123456799}]=2C "ClientSessions__": null=2C "ServerSessions__": null= =2C "Monetization__": null}
2011-03-03 18:00:00=2C208 INFO org.apache.ha= doop.mapred.TaskRunner: trace bug1 value is {"rowKey": "0000000000000000000= 000000000000000000"=2C "tableName": null=2C"Games__": [{"columnName": "0_TE= STFAM_TESTSKU_1.5"=2C "columnValue": {"bytes": "ame": "hwty"=2C "columnValu= e": "stringvalue"}=2C {"columnName": "loc"=2C "columnValue": "stringvalue"}= =2C {"columnName": "osrev"=2C "columnValue": "stringvalue"}=2C {"columnName= ": "tz"=2C "columnValue": "stringvalue"}]=2C "PlayerState__": [{"columnName= ":"0_TESTFAM_TESTSKU_1.0=3DGC=3D2010:01:01:07"=2C "columnValue": "{"mojo":1= 0=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}=2C {"columnNam= e":"0_TESTFAM_TESTSKU_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":= 10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456799}]=2C "ClientSe= ssions__": null=2C "ServerSessions__": null=2C "Monetization__": null}
2= 011-03-03 18:00:00=2C208 INFO org.apache.hadoop.mapred.Merger: trace bug ad= just priority queue
2011-03-03 18:00:00=2C208 INFO org.apache.avro.mapre= d.AvroSerialization: trace bug deserialize() reader org.apache.avro.specifi= c.SpecificDatumReader@1a001ff
2011-03-03 18:00:00=2C208 INFO org.apache.= avro.generic.GenericDatumReader: trace bug type of expected STRING
2011-= 03-03 18:00:00=2C209 INFO org.apache.avro.mapred.AvroSerialization: trace b= ug deserialized datum 0000000100000000000000000000000000001
2011-03-03 1= 8:00:00=2C209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserial= izer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1a= bcc03
2011-03-03 18:00:00=2C209 INFO org.apache.hadoop.mapred.TaskRunner= : trace bug1 key is 0000000100000000000000000000000000001
2011-03-03 18:= 00:00=2C210 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: wo= rking on 0000000000000000000000000000000000000 whose rowKey is 000000000000= 0000000000000000000000000
2011-03-03 18:00:00=2C215 INFO org.apache.hado= op.mapred.ReduceTask: trace bug call nextKey()
2011-03-03 18:00:00=2C215= INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 000000010000000= 0000000000000000000001 values is org.apache.hadoop.mapred.ReduceTask$Reduce= ValuesIterator@1deeb40
2011-03-03 18:00:00=2C215 INFO com.ngmoco.ngpipes= .sourcing.NgActivityGatheringReducer: work on key 0000000100000000000000000= 000000000001
2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroS= erialization: trace bug deserialize() reader org.apache.avro.specific.Speci= ficDatumReader@26e9f9
2011-03-03 18:00:00=2C216 INFO org.apache.avro.gen= eric.GenericDatumReader: trace bug type of expected RECORD
2011-03-03 18= :00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: trace bug deser= ialized datum {"rowKey": "0000000100000000000000000000000000001"=2C "tableN= ame":null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [{"columnName":= "0_TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":= 10=2C"afloat":1.99=2C"hat":"red"}"=2C"timestamp": 123456789}]=2C "ClientSes= sions__": null=2C "ServerSessions__": null=2C "Monetization__": null}
20= 11-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: trace bug= 1 value is {"rowKey": "0000000100000000000000000000000000001"=2C "tableName= ": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "0_= TESTFAM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null}
201= 1-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.Merger: trace bug adju= st priority queue
2011-03-03 18:00:00=2C216 INFO org.apache.avro.mapred.= AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.= SpecificDatumReader@1a001ff
2011-03-03 18:00:00=2C216 INFO org.apache.av= ro.generic.GenericDatumReader: trace bug type of expected STRING
2011-03= -03 18:00:00=2C216 INFO org.apache.avro.mapred.AvroSerialization: trace bug= deserialized datum 0000000200000000000000000000000000002
2011-03-03 18:= 00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializ= er is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abc= c03
2011-03-03 18:00:00=2C216 INFO org.apache.hadoop.mapred.TaskRunner: = trace bug1 key is 0000000200000000000000000000000000002
2011-03-03 18:00= :00=2C216 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work= ing on 0000000100000000000000000000000000001 whose rowKey is 00000002000000= 00000000000000000000002
2011-03-03 18:00:00=2C217 INFO org.apache.hadoop= .mapred.ReduceTask: trace bug call nextKey()
2011-03-03 18:00:00=2C217 I= NFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 00000002000000000= 00000000000000000002 values is org.apache.hadoop.mapred.ReduceTask$ReduceVa= luesIterator@1deeb40
2011-03-03 18:00:00=2C217 INFO com.ngmoco.ngpipes.s= ourcing.NgActivityGatheringReducer: work on key 000000020000000000000000000= 0000000002
2011-03-03 18:00:00=2C217 INFO org.apache.avro.mapred.AvroSer= ialization: trace bug deserialize() reader org.apache.avro.specific.Specifi= cDatumReader@26e9f9
2011-03-03 18:00:00=2C217 INFO org.apache.avro.gener= ic.GenericDatumReader: trace bug type of expected RECORD
2011-03-03 18:0= 0:00=2C218 INFO org.apache.avro.mapred.AvroSerialization: trace bug deseria= lized datum {"rowKey": "0000000200000000000000000000000000002"=2C "tableNam= e": null=2C "Gam=2C "Metadata__": null=2C "PlayerState__": [{"columnName": = "0_TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":1= 0=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSes= sions__": null=2C "ServerSessions__": null=2C "Monetization__": null}
20= 11-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: trace bug= 1 value is {"rowKey": "0000000200000000000000000000000000002"=2C "tableName= ": null=2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "0_= TESTFAM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10= =2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSess= ions__": null=2C "ServerSessions__": null=2C "Monetization__": null}
201= 1-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.Merger: trace bug adju= st priority queue
2011-03-03 18:00:00=2C218 INFO org.apache.avro.mapred.= AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.= SpecificDatumReader@1a001ff
2011-03-03 18:00:00=2C218 INFO org.apache.av= ro.generic.GenericDatumReader: trace bug type of expected STRING
2011-03= -03 18:00:00=2C218 INFO org.apache.avro.mapred.AvroSerialization: trace bug= deserialized datum 0000000300000000000000000000000000003
2011-03-03 18:= 00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializ= er is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abc= c03
2011-03-03 18:00:00=2C218 INFO org.apache.hadoop.mapred.TaskRunner: = trace bug1 key is 0000000300000000000000000000000000003
2011-03-03 18:00= :00=2C218 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work= ing on 0000000200000000000000000000000000002 whose rowKey is 00000003000000= 00000000000000000000003
 =3B =3B
=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D
log fragment without the statement 'record.put("rowkey"=2C key)
2011-03-03 21:02:05=2C077 INFO org.apache.hadoop.mapred.ReduceTask: tr= ace bug isSkipping():false
2011-03-03 21:02:05=2C092 INFO org.apache.avr= o.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.= specific.SpecificDatumReader@1a001ff
2011-03-03 21:02:05=2C102 INFO org.= apache.avro.generic.GenericDatumReader: trace bug type of expected STRING2011-03-03 21:02:05=2C102 INFO org.apache.avro.mapred.AvroSerialization: = trace bug deserialized datum 0000000000000000000000000000000000000
2011-= 03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 d= eserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserial= izer@1abcc03
2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred.Tas= kRunner: trace bug1 key is 0000000000000000000000000000000000000
2011-03= -03 21:02:05=2C102 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done= with set values
2011-03-03 21:02:05=2C102 INFO org.apache.hadoop.mapred= .ReduceTask: trace bug key is 0000000000000000000000000000000000000 values = is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40
2011= -03-03 21:02:05=2C103 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringR= educer: work on key 0000000000000000000000000000000000000
2011-03-03 21:= 02:05=2C103 INFO org.apache.avro.mapred.AvroSerialization: trace bug deseri= alize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9
2011-= 03-03 21:02:05=2C113 INFO org.apache.avro.generic.GenericDatumReader: trace= bug type of expected RECORD
2011-03-03 21:02:05=2C114 INFO org.apache.a= vro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000= 000000000000000000000000000000000"=2C "tableName": null=2C "Games__": [{"co= lumnName": "0_TESTFAM_TESTSKU_1.5"=2C "columnValue": {"bytes": "ame": "hwty= "=2C "columnValue": "stringvalue"}=2C {"columnName": "loc"=2C "columnValue"= : "stringvalue"}=2C {"columnName": "osrev"=2C "columnValue": "stringvalue"}= =2C {"columnName": "tz"=2C "columnValue": "stringvalue"}]=2C "PlayerState__= ": [{"columnName": "0_TESTFAM_TESTSKU_1.0=3DGC=3D2010:01:01:07"=2C "columnV= alue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 12345678= 9}=2C {"columnName": "0_TESTFAM_TESTSKU_1.0=3DGS=3D2010:01:01:07"=2C "colum= nValue": "{"mojo":10=2C"afloat":1.99=2C"hat":"red"}"=2C "timestamp": 123456= 799}]=2C "ClientSessions__": null=2C "ServerSessions__": null=2C "Monetizat= ion__": null}
2011-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.Ta= skRunner: trace bug1 value is {"rowKey": "000000000000000000000000000000000= 0000"=2C "tableName": null=2C "Games__": [{"columnName": "0_TESTFAM_TESTSKU= _1.5"=2C "columnValue": {"bytes": "ame": "hwty"=2C "columnValue": "stringva= lue"}=2C {"columnName": "loc"=2C "columnValue": "stringvalue"}=2C {"columnN= ame": "osrev"=2C "columnValue": "stringvalue"}=2C {"columnName": "tz"=2C "c= olumnValue": "stringvalue"}]=2C "PlayerState__": [{"columnName": "0_TESTFAM= _TESTSKU_1.0=3DGC=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloat"= :1.99=2C"hat":"red"}"=2C "timestamp": 123456789}=2C {"columnName": "0_TESTF= AM_TESTSKU_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloa= t":1.99=2C"hat":"red"}"=2C "timestamp": 123456799}]=2C "ClientSessions__": = null=2C "ServerSessions__": null=2C "Monetization__": null}
2011-03-03 2= 1:02:05=2C114 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priori= ty queue
2011-03-03 21:02:05=2C114 INFO org.apache.avro.mapred.AvroSeria= lization: trace bug deserialize() reader org.apache.avro.specific.SpecificD= atumReader@1a001ff
2011-03-03 21:02:05=2C114 INFO org.apache.avro.generi= c.GenericDatumReader: trace bug type of expected STRING
2011-03-03 21:02= :05=2C114 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserial= ized datum 0000000100000000000000000000000000001
2011-03-03 21:02:05=2C1= 14 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org= .apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03
20= 11-03-03 21:02:05=2C114 INFO org.apache.hadoop.mapred.TaskRunner: trace bug= 1 key is 0000000100000000000000000000000000001
2011-03-03 21:02:05=2C115= INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 00= 00000000000000000000000000000000000 whose rowKey is 00000000000000000000000= 00000000000000
2011-03-03 21:02:05=2C121 INFO org.apache.hadoop.mapred.R= educeTask: trace bug call nextKey()
2011-03-03 21:02:05=2C121 INFO org.a= pache.hadoop.mapred.ReduceTask: trace bug key is 00000001000000000000000000= 00000000001 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesItera= tor@1deeb40
2011-03-03 21:02:05=2C121 INFO com.ngmoco.ngpipes.sourcing.N= gActivityGatheringReducer: work on key 000000010000000000000000000000000000= 1
2011-03-03 21:02:05=2C121 INFO org.apache.avro.mapred.AvroSerializatio= n: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumRea= der@26e9f9
2011-03-03 21:02:05=2C121 INFO org.apache.avro.generic.Generi= cDatumReader: trace bug type of expected RECORD
2011-03-03 21:02:05=2C12= 1 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized dat= um {"rowKey": "0000000100000000000000000000000000001"=2C "tableName": null= =2C "Gam=2C "Metadata__": null=2C "PlayerState__": [{"columnName": "0_TESTF= AM2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afl= oat":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}]=2C "ClientSessions__"= : null=2C "ServerSessions__": null=2C "Monetization__": null}
2011-03-03= 21:02:05=2C121 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value = is {"rowKey": "0000000100000000000000000000000000001"=2C "tableName": null= =2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "0_TESTFAM= 2_TESTSKU2_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloa= t":1.99=2C"hat":"red"}"=2C "timestamp": 123456789}]=2C "ClientSessions__": = null=2C "ServerSessions__": null=2C "Monetization__": null}
2011-03-03 2= 1:02:05=2C122 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priori= ty queue
2011-03-03 21:02:05=2C122 INFO org.apache.avro.mapred.AvroSeria= lization: trace bug deserialize() reader org.apache.avro.specific.SpecificD= atumReader@1a001ff
2011-03-03 21:02:05=2C122 INFO org.apache.avro.generi= c.GenericDatumReader: trace bug type of expected STRING
2011-03-03 21:02= :05=2C122 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserial= ized datum 0000000200000000000000000000000000002
2011-03-03 21:02:05=2C1= 22 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org= .apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03
20= 11-03-03 21:02:05=2C122 INFO org.apache.hadoop.mapred.TaskRunner: trace bug= 1 key is 0000000200000000000000000000000000002
2011-03-03 21:02:05=2C122= INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 00= 00000100000000000000000000000000001 whose rowKey is 00000001000000000000000= 00000000000001
2011-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.R= educeTask: trace bug call nextKey()
2011-03-03 21:02:05=2C123 INFO org.a= pache.hadoop.mapred.ReduceTask: trace bug key is 00000002000000000000000000= 00000000002 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesItera= tor@1deeb40
2011-03-03 21:02:05=2C123 INFO com.ngmoco.ngpipes.sourcing.N= gActivityGatheringReducer: work on key 000000020000000000000000000000000000= 2
2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSerializatio= n: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumRea= der@26e9f9
2011-03-03 21:02:05=2C123 INFO org.apache.avro.generic.Generi= cDatumReader: trace bug type of expected RECORD
2011-03-03 21:02:05=2C12= 3 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized dat= um {"rowKey": "0000000200000000000000000000000000002"=2C "tableName": null= =2C "Gam=2C "Metadata__": null=2C "PlayerState__": [{"columnName": "0_TESTF= AM3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afl= oat":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSessions__"= : null=2C "ServerSessions__": null=2C "Monetization__": null}
2011-03-03= 21:02:05=2C123 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value = is {"rowKey": "0000000200000000000000000000000000002"=2C "tableName": null= =2C "Games__": [{"colu: null=2C "PlayerState__": [{"columnName": "0_TESTFAM= 3_TESTSKU3_1.0=3DGS=3D2010:01:01:07"=2C "columnValue": "{"mojo":10=2C"afloa= t":1.99=2C"hat":"red"}"=2C "timestamp": 123456899}]=2C "ClientSessions__": = null=2C "ServerSessions__": null=2C "Monetization__": null}
2011-03-03 2= 1:02:05=2C123 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priori= ty queue
2011-03-03 21:02:05=2C123 INFO org.apache.avro.mapred.AvroSeria= lization: trace bug deserialize() reader org.apache.avro.specific.SpecificD= atumReader@1a001ff
2011-03-03 21:02:05=2C123 INFO org.apache.avro.generi= c.GenericDatumReader: trace bug type of expected STRING
2011-03-03 21:02= :05=2C123 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserial= ized datum 0000000300000000000000000000000000003
2011-03-03 21:02:05=2C1= 23 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org= .apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03
20= 11-03-03 21:02:05=2C123 INFO org.apache.hadoop.mapred.TaskRunner: trace bug= 1 key is 0000000300000000000000000000000000003
2011-03-03 21:02:05=2C124= INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 00= 00000200000000000000000000000000002 whose rowKey is 00000002000000000000000= 00000000000002


From: eychih@hotmail.com
To: user@avro.apache.org
Subject: RE: is this a bug?
Dat= e: Wed=2C 2 Mar 2011 16:12:20 -0800

Sorry I found that my previous message in the archive become all in black. =  =3BLet me re-explain the problem.  =3BThe following piece of code = for AvroReducer causes problem:

 =3B =3B  = =3B  =3B  =3B  =3B public void reduce(Utf8 key=2C Iterable<= =3BGenericRecord>=3B values=2C AvroCollector<=3BGenericRecord>=3B col= lector=2C Reporter reporter) throws IOException {
 =3B = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B  =3B  =3B GenericRecord record =3D null=3B
&nbs= p=3B =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B  =3B  =3B  =3B for (GenericRecord value : values) {
 =3B =3B  =3B  =3B  =3B  =3B  =3B &nbs= p=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B -- code omitted here --
 =3B =3B  =3B &n= bsp=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B record =3D value=3B
<= div> =3B =3B  =3B  =3B  =3B  =3B  =3B  =3B =  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B &nb= sp=3B record.put("rowkey"=2C key)=3B  =3B <=3B=3D=3D=3D this statemen= t causes problem
 =3B =3B  =3B  =3B  =3B &nbs= p=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B collector.collect(record)=3B
 = =3B =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B =  =3B  =3B  =3B  =3B }
 =3B =3B  =3B &= nbsp=3B  =3B  =3B  =3B}

As explained i= n my previous message=2C if I remove the statement record.put("rowkey"=2C k= ey)=2C the code works fine=2C in that the key values pairs passed to the ro= utine reduce() are correct.  =3BBut if you add this statement=2C the ke= y values pairs passed to the routine reduce() are out of order=2C something= like (key1=2C values1)=2C (key2=2C values3) rather than (key2=2C values2).=  =3BSome details are explained in my previous message.  =3BIs &nbs= p=3Bthis problem relating to Hadoop binary iterators or Avro deserializatio= n code?  =3BThanks.

Ey-Chih Chow

From: eyc= hih@hotmail.com
To: user@avr= o.apache.org
Subject: is this a bug?
Date: Wed=2C 2 Mar 2011 13:0= 5:55 -0800

Hi=2C

I am working on an Avro MR job and encountering an= issue with AvroReducer<=3BUtf8=2C GenericRecord=2C GenericRecord>=3B. = The corresponding reduce() routine is implemented in the following way:

public void reduce(Utf8 key=2C Iterable<=3BGenericR= ecord>=3B values=2C AvroCollector<=3BGenericRecord>=3B collector=2C R= eporter reporter) throws IOException {

 =3B =3B &n= bsp=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B.
 =3B =3B  =3B  =3B  =3B  =3B  =3B  =3B &= nbsp=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B &nbs= p=3B  =3B.
 =3B =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B  =3B  =3B  =3B.

 =3B =3B &nb= sp=3B  =3B GenericRecord record =3D null=3B

 =3B&n= bsp=3B  =3B  =3B for (GenericRecord value : values) {
&nb= sp=3B =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B .
 =3B =3B  =3B  =3B  =3B  =3B =  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B &nb= sp=3B  =3B  =3B  =3B .
 =3B =3B  =3B &nbs= p=3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B .
 = =3B =3B  =3B  =3B  =3B  =3B  =3Brecord =3D value=3B=
 =3B =3B  =3B  =3B  =3B  =3B  =3Brecord.put("rowkey"=2C k= ey)=3B
 =3B =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B  =3B  =3B  =3B .
 =3B =3B  =3B =  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B &nb= sp=3B  =3B  =3B  =3B  =3B  =3B  =3B .
&nb= sp=3B =3B  =3B  =3B  =3B  =3B  =3B  =3B  = =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B  =3B=  =3B .
 =3B =3B  =3B  =3B  =3B  =3B =  =3Bcollector.collect(record)=3B
 =3B =3B  =3B &n= bsp=3B  =3B }
} =3B

If I comment out the= statement in red in the above code=2C the reduce function gets called prop= erly with CORRECT key values pairs passed to reduce().  =3BHowever=2C i= f I add the statement in red to the routine=2C the reduce function is calle= d with WRONG key values pairs=2C in the sense that key2 paired with values3= =2C instead of values2=2C when passed to the reduce() routine.  =3BI tr= aced this problem by including Hadoop source code=2C such as ReduceTask.jav= a=2C Task.java=2C and Avro source code=2C such as HadoopReducer.java=2C Had= oopReducerBase.java=2C and all the serialization code.  =3BThe problem = showed up on the second call of the reduce()=2C but I can not locate the ex= act place that cause the problem.  =3BMy intuition is that this is incu= rred in either the hadoop iterators after merge sort or Avro deserializatio= n.  =3BIs there anybody can help me on this?  =3BThanks.
=
Ey-Chih Chow  =3B  =3B  =3B
=
= --_885d0a80-99ff-4cdb-b101-272536beb62a_--