|
ey-chih chow
2011-03-02, 21:05
ey-chih chow
2011-03-03, 00:12
ey-chih chow
2011-03-04, 08:48
ey-chih chow
2011-03-05, 00:57
ey-chih chow
2011-03-08, 22:35
Scott Carey
2011-03-08, 23:06
ey-chih chow
2011-03-10, 19:33
ey-chih chow
2011-03-10, 22:28
ey-chih chow
2011-03-10, 23:16
Scott Carey
2011-03-10, 23:53
Scott Carey
2011-03-11, 00:01
ey-chih chow
2011-03-16, 23:59
Scott Carey
2011-03-17, 00:06
|
-
is this a bug?ey-chih chow 2011-03-02, 21:05
Hi, I am working on an Avro MR job and encountering an issue with AvroReducer<Utf8, GenericRecord, GenericRecord>. The corresponding reduce() routine is implemented in the following way: public void reduce(Utf8 key, Iterable<GenericRecord> values, AvroCollector<GenericRecord> collector, Reporter reporter) throws IOException { . . . GenericRecord record = null; for (GenericRecord value : values) { . . . record = value; record.put("rowkey", key); . . . collector.collect(record); }} If I comment out the statement in red in the above code, the reduce function gets called properly with CORRECT key values pairs passed to reduce(). However, if I add the statement in red to the routine, the reduce function is called with WRONG key values pairs, in the sense that key2 paired with values3, instead of values2, when passed to the reduce() routine. I traced this problem by including Hadoop source code, such as ReduceTask.java, Task.java, and Avro source code, such as HadoopReducer.java, HadoopReducerBase.java, and all the serialization code. The problem showed up on the second call of the reduce(), but I can not locate the exact 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
-
RE: is this a bug?ey-chih chow 2011-03-03, 00:12
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 AvroReducer causes problem: public void reduce(Utf8 key, Iterable<GenericRecord> values, AvroCollector<GenericRecord> collector, Reporter reporter) throws IOException { GenericRecord record = null; for (GenericRecord value : values) { -- code omitted here -- record = value; record.put("rowkey", key); <=== this statement causes problem collector.collect(record); } } As explained in my previous message, if I remove the statement record.put("rowkey", key), the code works fine, in that the key values pairs passed to the routine reduce() are correct. But if you add this statement, the key values pairs passed to the routine reduce() are out of order, something like (key1, values1), (key2, values3) rather than (key2, values2). Some details are explained in my previous message. Is this problem relating to Hadoop binary iterators or Avro deserialization code? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: is this a bug? Date: Wed, 2 Mar 2011 13:05:55 -0800 Hi, I am working on an Avro MR job and encountering an issue with AvroReducer<Utf8, GenericRecord, GenericRecord>. The corresponding reduce() routine is implemented in the following way: public void reduce(Utf8 key, Iterable<GenericRecord> values, AvroCollector<GenericRecord> collector, Reporter reporter) throws IOException { . . . GenericRecord record = null; for (GenericRecord value : values) { . . . record = value; record.put("rowkey", key); . . . collector.collect(record); }} If I comment out the statement in red in the above code, the reduce function gets called properly with CORRECT key values pairs passed to reduce(). However, if I add the statement in red to the routine, the reduce function is called with WRONG key values pairs, in the sense that key2 paired with values3, instead of values2, when passed to the reduce() routine. I traced this problem by including Hadoop source code, such as ReduceTask.java, Task.java, and Avro source code, such as HadoopReducer.java, HadoopReducerBase.java, and all the serialization code. The problem showed up on the second call of the reduce(), but I can not locate the exact 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
-
RE: is this a bug?ey-chih chow 2011-03-04, 08:48
What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected RECORD 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000000000000000000000000000000000000", "tableName": null, "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName": "0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value is {"rowKey": "0000000000000000000000000000000000000", "tableName": null,"Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName":"0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName":"0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priority queue 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,209 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000100000000000000000000000000001 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000100000000000000000000000000001 2011-03-03 18:00:00,210 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 2011-03-03 18:00:00,215 INFO org.apache.hadoop.mapred.ReduceTask: trace bug call nextKey() 2011-03-03 18:00:00,215 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000100000000000000000000000000001 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,215 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000100000000000000000000000000001 2011-03-03 18:00:00,216 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9 2011-03-03 18:00:00,216 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected RECORD 2011-03-03 18:00:00,216 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000000100000000000000000000000000001", "tableName":null, "
-
RE: is this a bug?ey-chih chow 2011-03-05, 00:57
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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected RECORD 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000000000000000000000000000000000000", "tableName": null, "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName": "0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value is {"rowKey": "0000000000000000000000000000000000000", "tableName": null,"Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName":"0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName":"0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priority queue 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,209 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000100000000000000000000000000001 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000100000000000000000000000000001 2011-03-03 18:00:00,210 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 2011-03-03 18:00:00,215 INFO org.apache.hadoop.mapred.ReduceTask: trace bug call nextKey() 2011-03-03 18:00:00,215 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000100000000000000000000000000001 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,215 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000100000000000000000000000000001 2011-03-03 18:00:00,216 IN
-
RE: is this a bug?ey-chih chow 2011-03-08, 22:35
Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 in all of our code. Some of the jobs show weird behavior. We want to know if this is fixable. Otherwise, we have to take out all the avro APIs and use the conventional MR APIs instead. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected RECORD 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000000000000000000000000000000000000", "tableName": null, "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName": "0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value is {"rowKey": "0000000000000000000000000000000000000", "tableName": null,"Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName":"0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName":"0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priority queue 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,209 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000100000000000000000000000000001 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,209 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000100000000000000000000000000001 2011-03-03 18:00:00,210 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 2011-03-03 18:00:00,215 INFO org.apache.hadoop.ma
-
Re: is this a bug?Scott Carey 2011-03-08, 23:06
I haven't completely gone through your messages to understand your problem completely. However, there were a couple fixes 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: https://repository.apache.org/content/repositories/orgapacheavro-001/ release candidate: http://people.apache.org/~cutting/avro-1.5.0-rc3/ Note there are some API changes that may affect you a little, see CHANGES.txt -Scott On 3/8/11 2:35 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 in all of our code. Some of the jobs show weird behavior. We want to know if this is fixable. Otherwise, we have to take out all the avro APIs and use the conventional MR APIs instead. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Fri, 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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@26e9f9 2011-03-03 18:00:00,208 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected RECORD 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum {"rowKey": "0000000000000000000000000000000000000", "tableName": null, "Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName": "0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName": "0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 value is {"rowKey": "0000000000000000000000000000000000000", "tableName": null,"Games__": [{"columnName": "0_TESTFAM_TESTSKU_1.5", "columnValue": {"bytes": "ame": "hwty", "columnValue": "stringvalue"}, {"columnName": "loc", "columnValue": "stringvalue"}, {"columnName": "osrev", "columnValue": "stringvalue"}, {"columnName": "tz", "columnValue": "stringvalue"}], "PlayerState__": [{"columnName":"0_TESTFAM_TESTSKU_1.0=GC=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456789}, {"columnName":"0_TESTFAM_TESTSKU_1.0=GS=2010:01:01:07", "columnValue": "{"mojo":10,"afloat":1.99,"hat":"red"}", "timestamp": 123456799}], "ClientSessions__": null, "ServerSessions__": null, "Monetization__": null} 2011-03-03 18:00:00,208 INFO org.apache.hadoop.mapred.Merger: trace bug adjust priority queue 2011-03-03 18:00:00,208 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,208 INFO or
-
RE: is this a bug?ey-chih chow 2011-03-10, 19:33
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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>, null} Games__; The error messages are as follows: 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_20110310113041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbase.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.DeviceRow 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.serialize(AvroSerialization.java:131) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.serialize(AvroSerialization.java:114) at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:900) at org.apache.hadoop.mapred.MapTask$OldOutputCollector.collect(MapTask.java:466) at org.apache.avro.mapred.HadoopMapper$MapCollector.collect(HadoopMapper.java:69) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.java:91) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.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.java:92) at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:86) at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWriter.java:102) ... 14 more From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 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, there were a couple fixes 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: https://repository.apache.org/content/repositories/orgapacheavro-001/release candidate: http://people.apache.org/~cutting/avro-1.5.0-rc3/ Note there are some API changes that may affect you a little, see CHANGES.txt -Scott On 3/8/11 2:35 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 in all of our code. Some of the jobs show weird behavior. We want to know if this is fixable. Otherwise, we have to take out all the avro APIs and use the conventional MR APIs instead. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 values is org.apache.hadoop.mapred.ReduceTask$ReduceValuesIterator@1deeb40 2011-03-03 18:00:00,199 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: work on key 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader
-
RE: is this a bug?ey-chih chow 2011-03-10, 22:28
I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Thu, 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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>, null} Games__; The error messages are as follows: 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_20110310113041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbase.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.DeviceRow 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.serialize(AvroSerialization.java:131) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.serialize(AvroSerialization.java:114) at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:900) at org.apache.hadoop.mapred.MapTask$OldOutputCollector.collect(MapTask.java:466) at org.apache.avro.mapred.HadoopMapper$MapCollector.collect(HadoopMapper.java:69) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.java:91) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.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.java:92) at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:86) at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWriter.java:102) ... 14 more From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 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, there were a couple fixes 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: https://repository.apache.org/content/repositories/orgapacheavro-001/release candidate: http://people.apache.org/~cutting/avro-1.5.0-rc3/ Note there are some API changes that may affect you a little, see CHANGES.txt -Scott On 3/8/11 2:35 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 in all of our code. Some of the jobs show weird behavior. We want to know if this is fixable. Otherwise, we have to take out all the avro APIs and use the conventional MR APIs instead. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialized datum 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 deserializer is org.apache.avro.mapred.AvroSerialization$AvroWrapperDeserializer@1abcc03 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.TaskRunner: trace bug1 key is 0000000000000000000000000000000000000 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug done with set values 2011-03-03 18:00:00,199 INFO org.apache.hadoop.mapred.ReduceTask: trace bug key is 0000000000000000000000000000000000000 val
-
RE: is this a bug?ey-chih chow 2011-03-10, 23:16
After I made the change mentioned in the previous message, The MR job was running. However, this did not fix the problem I mentioned at the beginning of the topic. I got the following for the reducer: ====================================================================================================attempt_20110310145147365_0002_r_000000_0/syslog:2011-03-10 14:52:31,226 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,010 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,016 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000200000000000000000000000000002attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,017 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,021 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000400000000000000000000000000004attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,023 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000500000000000000000000000000005attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,024 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005===================================================================================================If we add the following two lines to the reducer code: ====================================================================================================boolean workAround = getConf().getBoolean(NgActivityGatheringJob.NG_AVRO_BUG_WORKAROUND, true);Utf8 dupKey = (workAround) ? new Utf8(key.toString()) : key; // use dupKey instead of key passed to reducer===================================================================================================We got the following trace, which we consider as the right behavior: ====================================================================================================2011-03-10 15:04:33,431 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,374 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,381 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000100000000000000000000000000001attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,383 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,389 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000300000000000000000000000000003attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,391 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000400000000000000000000000000004attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,393 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005=================================================================================================== Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Thu, 10 Mar 2011 14:28:41 -0800 I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Thu, 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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>, null} Games__; The error messages are as follows: 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_20110310113041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbase.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.DeviceRow 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.serialize(AvroSerialization.java:131) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.serialize(AvroSerialization.java:114) at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:900) at org.apache.hadoop.mapred.MapTask$OldOutputCollector.collect(MapTask.java:466) at org.apache.avro.mapred.HadoopMapper$MapCollector.collect(HadoopMapper
-
Re: is this a bug?Scott Carey 2011-03-10, 23:53
This might be a bug in Union handling in ReflectDatumWriter, can you file a bug in JIRA?
Thanks, -Scott On 3/10/11 2:28 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Thu, 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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>, null} Games__; The error messages are as follows: 11/03/10 11:31:02 INFO mapred.TaskInProgress: Error from attempt_20110310113041953_0001_m_000000_0: java.lang.NullPointerException: in com.ngmoco.hbase.DeviceRow in union null of union in field Games__ of com.ngmoco.hbase.DeviceRow 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.serialize(AvroSerialization.java:131) at org.apache.avro.mapred.AvroSerialization$AvroWrapperSerializer.serialize(AvroSerialization.java:114) at org.apache.hadoop.mapred.MapTask$MapOutputBuffer.collect(MapTask.java:900) at org.apache.hadoop.mapred.MapTask$OldOutputCollector.collect(MapTask.java:466) at org.apache.avro.mapred.HadoopMapper$MapCollector.collect(HadoopMapper.java:69) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.java:91) at com.ngmoco.ngpipes.sourcing.NgActivityGatheringMapper.map(NgActivityGatheringMapper.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.java:92) at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:86) at org.apache.avro.reflect.ReflectDatumWriter.write(ReflectDatumWriter.java:102) ... 14 more ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 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, there were a couple fixes 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: https://repository.apache.org/content/repositories/orgapacheavro-001/ release candidate: http://people.apache.org/~cutting/avro-1.5.0-rc3/ Note there are some API changes that may affect you a little, see CHANGES.txt -Scott On 3/8/11 2:35 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: Can anybody tell me if this this a bug? We use avro map/reduce API v 1.4 in all of our code. Some of the jobs show weird behavior. We want to know if this is fixable. Otherwise, we have to take out all the avro APIs and use the conventional MR APIs instead. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Fri, 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 record.put("rowkey", key) in the reduce() method. Does this mean that there is a bug in BinaryDecoder.java ? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Fri, 4 Mar 2011 00:48:55 -0800 What follows are fragments of trace logs of our MR jobs corresponding respectively to with and without the statement 'record.put("rowkey", key)' mentioned in the previous messages. From the last line, logged at the entry of the reduce() method, of each of these two logs you can see the difference. I.e. for the first segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003' for the second segment, the log is 'working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002', where the second log is what we expected, 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 and Avro source code. Can anybody help to see if this is a bug in Avro or Hadoop code? ============================================================================================================= log fragment with the statement 'record.put("rowkey", key) 2011-03-03 18:00:00,180 INFO org.apache.hadoop.mapred.ReduceTask: trace bug isSkipping():false 2011-03-03 18:00:00,190 INFO org.apache.avro.mapred.AvroSerialization: trace bug deserialize() reader org.apache.avro.specific.SpecificDatumReader@1a001ff 2011-03-03 18:00:00,198 INFO org.apache.avro.generic.GenericDatumReader: trace bug type of expected STRING 2011-03-03 18:00:00,199 INFO org.apache.avro.mapred.AvroSerializatio
-
Re: is this a bug?Scott Carey 2011-03-11, 00:01
One thing that could be related is that Hadoop under the covers re-uses objects, so modifying one returned by reduce() and passing it on may not behave as expected. Your work-around below seems to indicate that that it may be related to object re-use. The easiest way for us to figure this out is to have a reproducible use case. If you can provide a patch that adds a Unit test to Avro that fails, that would help greatly. The Unit test would probably be the easiest on your end, since we already have M/R Unit tests in Avro that do most of the work of configuring and running a simple M/R job. -Scott On 3/10/11 3:16 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: After I made the change mentioned in the previous message, The MR job was running. However, this did not fix the problem I mentioned at the beginning of the topic. I got the following for the reducer: ===================================================================================================attempt_20110310145147365_0002_r_000000_0/syslog:2011-03-10 14:52:31,226 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,010 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,016 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000200000000000000000000000000002 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,017 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,021 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000400000000000000000000000000004 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,023 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000500000000000000000000000000005 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,024 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005 =================================================================================================== If we add the following two lines to the reducer code: ===================================================================================================boolean workAround = getConf().getBoolean(NgActivityGatheringJob.NG_AVRO_BUG_WORKAROUND, true); Utf8 dupKey = (workAround) ? new Utf8(key.toString()) : key; // use dupKey instead of key passed to reducer =================================================================================================== We got the following trace, which we consider as the right behavior: ===================================================================================================2011-03-10 15:04:33,431 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,374 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,381 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000100000000000000000000000000001 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,383 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,389 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000300000000000000000000000000003 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,391 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000400000000000000000000000000004 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,393 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005 =================================================================================================== Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Thu, 10 Mar 2011 14:28:41 -0800 I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Thu, 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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>, null} Games__; The erro
-
RE: is this a bug?ey-chih chow 2011-03-16, 23:59
I have created a Unit test that can reproduce the problem. Do you want me to file a bug for this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 10 Mar 2011 16:01:39 -0800 Subject: Re: is this a bug? One thing that could be related is that Hadoop under the covers re-uses objects, so modifying one returned by reduce() and passing it on may not behave as expected. Your work-around below seems to indicate that that it may be related to object re-use. The easiest way for us to figure this out is to have a reproducible use case. If you can provide a patch that adds a Unit test to Avro that fails, that would help greatly. The Unit test would probably be the easiest on your end, since we already have M/R Unit tests in Avro that do most of the work of configuring and running a simple M/R job. -Scott On 3/10/11 3:16 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: After I made the change mentioned in the previous message, The MR job was running. However, this did not fix the problem I mentioned at the beginning of the topic. I got the following for the reducer: ====================================================================================================attempt_20110310145147365_0002_r_000000_0/syslog:2011-03-10 14:52:31,226 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,010 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,016 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000200000000000000000000000000002attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,017 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,021 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000400000000000000000000000000004attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,023 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000500000000000000000000000000005attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,024 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005===================================================================================================If we add the following two lines to the reducer code: ====================================================================================================boolean workAround = getConf().getBoolean(NgActivityGatheringJob.NG_AVRO_BUG_WORKAROUND, true);Utf8 dupKey = (workAround) ? new Utf8(key.toString()) : key; // use dupKey instead of key passed to reducer===================================================================================================We got the following trace, which we consider as the right behavior: ====================================================================================================2011-03-10 15:04:33,431 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,374 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,381 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000100000000000000000000000000001attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,383 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,389 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000300000000000000000000000000003attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,391 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000400000000000000000000000000004attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,393 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005=================================================================================================== Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Thu, 10 Mar 2011 14:28:41 -0800 I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: is this a bug? Date: Thu, 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 should change? Our avdl record, DeviceRow, has a field defined as follows: union {array<DynamicColumn4Games>,
-
Re: is this a bug?Scott Carey 2011-03-17, 00:06
Yes, That would be great!
Thanks, -Scott On 3/16/11 4:59 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I have created a Unit test that can reproduce the problem. Do you want me to file a bug for this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 10 Mar 2011 16:01:39 -0800 Subject: Re: is this a bug? One thing that could be related is that Hadoop under the covers re-uses objects, so modifying one returned by reduce() and passing it on may not behave as expected. Your work-around below seems to indicate that that it may be related to object re-use. The easiest way for us to figure this out is to have a reproducible use case. If you can provide a patch that adds a Unit test to Avro that fails, that would help greatly. The Unit test would probably be the easiest on your end, since we already have M/R Unit tests in Avro that do most of the work of configuring and running a simple M/R job. -Scott On 3/10/11 3:16 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: After I made the change mentioned in the previous message, The MR job was running. However, this did not fix the problem I mentioned at the beginning of the topic. I got the following for the reducer: ===================================================================================================attempt_20110310145147365_0002_r_000000_0/syslog:2011-03-10 14:52:31,226 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,010 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,016 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000200000000000000000000000000002 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,017 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000300000000000000000000000000003 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,021 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000400000000000000000000000000004 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,023 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000500000000000000000000000000005 attempt_20110310145315542_0002_r_000000_0/syslog:2011-03-10 14:53:59,024 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005 =================================================================================================== If we add the following two lines to the reducer code: ===================================================================================================boolean workAround = getConf().getBoolean(NgActivityGatheringJob.NG_AVRO_BUG_WORKAROUND, true); Utf8 dupKey = (workAround) ? new Utf8(key.toString()) : key; // use dupKey instead of key passed to reducer =================================================================================================== We got the following trace, which we consider as the right behavior: ===================================================================================================2011-03-10 15:04:33,431 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,374 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000000000000000000000000000000000 whose rowKey is 0000000000000000000000000000000000000 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,381 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000100000000000000000000000000001 whose rowKey is 0000000100000000000000000000000000001 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,383 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000200000000000000000000000000002 whose rowKey is 0000000200000000000000000000000000002 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,389 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000300000000000000000000000000003 whose rowKey is 0000000300000000000000000000000000003 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,391 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000400000000000000000000000000004 whose rowKey is 0000000400000000000000000000000000004 attempt_20110310150517897_0002_r_000000_0/syslog:2011-03-10 15:06:01,393 INFO com.ngmoco.ngpipes.sourcing.NgActivityGatheringReducer: working on 0000000500000000000000000000000000005 whose rowKey is 0000000500000000000000000000000000005 =================================================================================================== Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: is this a bug? Date: Thu, 10 Mar 2011 14:28:41 -0800 I changed the Games__ field of the DeviceRow to union {null, array<DynamicColumn4Games>} Games__; the system seemed no longer complaining. Is this a right fix? Thanks. Ey-Chih Chow _______________ |