|
ey-chih chow
2011-05-31, 17:38
ey-chih chow
2011-06-01, 00:40
Scott Carey
2011-06-01, 01:26
ey-chih chow
2011-06-01, 18:34
Scott Carey
2011-06-01, 20:38
ey-chih chow
2011-06-01, 22:14
Scott Carey
2011-06-01, 22:27
ey-chih chow
2011-06-09, 22:08
ey-chih chow
2011-06-09, 22:16
ey-chih chow
2011-06-09, 22:26
Scott Carey
2011-06-09, 22:45
Scott Carey
2011-06-09, 22:42
ey-chih chow
2011-06-09, 23:45
Scott Carey
2011-06-09, 23:58
Scott Carey
2011-06-10, 00:43
ey-chih chow
2011-06-10, 17:53
Scott Carey
2011-06-10, 18:11
ey-chih chow
2011-06-10, 19:07
Scott Carey
2011-06-10, 22:58
Scott Carey
2011-06-01, 20:45
ey-chih chow
2011-06-01, 22:21
Tatu Saloranta
2011-06-02, 00:13
Scott Carey
2011-06-02, 00:45
Tatu Saloranta
2011-06-02, 00:48
Scott Carey
2011-06-02, 01:48
ey-chih chow
2011-06-02, 17:23
Scott Carey
2011-06-02, 21:14
|
-
avro object reuseey-chih chow 2011-05-31, 17:38
Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-05-31, 17:38
-
RE: avro object reuseey-chih chow 2011-06-01, 00:40
I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-06-01, 00:40
-
Re: avro object reuseScott Carey 2011-06-01, 01:26
All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues.
I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
Scott Carey 2011-06-01, 01:26
-
RE: avro object reuseey-chih chow 2011-06-01, 18:34
We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-06-01, 18:34
-
Re: avro object reuseScott Carey 2011-06-01, 20:38
This is great info.
Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
Scott Carey 2011-06-01, 20:38
-
RE: avro object reuseey-chih chow 2011-06-01, 22:14
We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-06-01, 22:14
-
Re: avro object reuseScott Carey 2011-06-01, 22:27
no, and even GenericData.Record simply writes using a StringBuilder; I doubt this is the culprit.
On 6/1/11 3:14 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
Scott Carey 2011-06-01, 22:27
-
RE: avro object reuseey-chih chow 2011-06-09, 22:08
We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-06-09, 22:08
-
RE: avro object reuseey-chih chow 2011-06-09, 22:16
I forgot to mention that the histogram in my previous message was extracted from a mapper of one of our MR job. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Thu, 9 Jun 2011 15:08:02 -0700 We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
ey-chih chow 2011-06-09, 22:16
-
RE: avro object reuseey-chih chow 2011-06-09, 22:26
In addition, we ran the same MR job once again and got the following histogram. Whey this is different from the previous one? Thanks. Ey-Chih Chow Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4327 100242096 byte[] 2: 2050 5381496 int[] 3: 23762 2822864 * ConstMethodKlass 4: 23762 1904760 * MethodKlass 5: 39295 1688992 * SymbolKlass 6: 2127 1216976 * ConstantPoolKlass 7: 2127 882760 * InstanceKlassKlass 8: 11298 773008 char[] 9: 1847 742936 * ConstantPoolCacheKlass 10: 1064 297448 * MethodDataKlass 11: 11387 273288 java.lang.String 12: 2317 222432 java.lang.Class 13: 3288 204440 short[] 14: 3167 156664 * System ObjArray 15: 1360 86720 java.util.HashMap$Entry[] 16: 535 85600 org.codehaus.jackson.impl.ReaderBasedParser 17: 3498 83952 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1267 44704 java.lang.Object[] 21: 1808 43392 java.util.Hashtable$Entry 22: 1070 42800 org.codehaus.jackson.impl.JsonReadContext 23: 777 31080 java.util.HashMap 24: 535 29960 org.codehaus.jackson.util.TextBuffer 25: 567 27216 java.nio.HeapByteBuffer 26: 553 26544 org.apache.avro.Schema$Props 27: 549 26352 java.nio.HeapCharBuffer 28: 538 25824 org.codehaus.jackson.map.DeserializationConfig 29: 535 25680 org.codehaus.jackson.io.IOContext 30: 1554 24864 org.codehaus.jackson.sym.CharsToNameCanonicalizer$Bucket 31: 539 21560 org.codehaus.jackson.sym.CharsToNameCanonicalizer From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Thu, 9 Jun 2011 15:16:29 -0700 I forgot to mention that the histogram in my previous message was extracted from a mapper of one of our MR job. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Thu, 9 Jun 2011 15:08:02 -0700 We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.code +
ey-chih chow 2011-06-09, 22:26
-
Re: avro object reuseScott Carey 2011-06-09, 22:45
If you do just 'jmap –histo' it shows you all of the objects on the heap. Many of these objects may be garbage and unreferenced. This is quick, and does not block the app or force a GC.
If you do 'jmap –histo:live' it will GC and only show the objects that are 'live' (currently referenced). These are different because a GC ran and removed all the BinaryData inner class temporary objects. On 6/9/11 3:26 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: In addition, we ran the same MR job once again and got the following histogram. Whey this is different from the previous one? Thanks. Ey-Chih Chow Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4327 100242096 byte[] 2: 2050 5381496 int[] 3: 23762 2822864 * ConstMethodKlass 4: 23762 1904760 * MethodKlass 5: 39295 1688992 * SymbolKlass 6: 2127 1216976 * ConstantPoolKlass 7: 2127 882760 * InstanceKlassKlass 8: 11298 773008 char[] 9: 1847 742936 * ConstantPoolCacheKlass 10: 1064 297448 * MethodDataKlass 11: 11387 273288 java.lang.String 12: 2317 222432 java.lang.Class 13: 3288 204440 short[] 14: 3167 156664 * System ObjArray 15: 1360 86720 java.util.HashMap$Entry[] 16: 535 85600 org.codehaus.jackson.impl.ReaderBasedParser 17: 3498 83952 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1267 44704 java.lang.Object[] 21: 1808 43392 java.util.Hashtable$Entry 22: 1070 42800 org.codehaus.jackson.impl.JsonReadContext 23: 777 31080 java.util.HashMap 24: 535 29960 org.codehaus.jackson.util.TextBuffer 25: 567 27216 java.nio.HeapByteBuffer 26: 553 26544 org.apache.avro.Schema$Props 27: 549 26352 java.nio.HeapCharBuffer 28: 538 25824 org.codehaus.jackson.map.DeserializationConfig 29: 535 25680 org.codehaus.jackson.io.IOContext 30: 1554 24864 org.codehaus.jackson.sym.CharsToNameCanonicalizer$Bucket 31: 539 21560 org.codehaus.jackson.sym.CharsToNameCanonicalizer ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Thu, 9 Jun 2011 15:16:29 -0700 I forgot to mention that the histogram in my previous message was extracted from a mapper of one of our MR job. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Thu, 9 Jun 2011 15:08:02 -0700 We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On +
Scott Carey 2011-06-09, 22:45
-
Re: avro object reuseScott Carey 2011-06-09, 22:42
The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC.
The below have only 32 bytes each and 8MB total. On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many t +
Scott Carey 2011-06-09, 22:42
-
RE: avro object reuseey-chih chow 2011-06-09, 23:45
We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total.On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will t +
ey-chih chow 2011-06-09, 23:45
-
Re: avro object reuseScott Carey 2011-06-09, 23:58
What is the stack trace on the out of memory exception?
On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total. On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usa +
Scott Carey 2011-06-09, 23:58
-
Re: avro object reuseScott Carey 2011-06-10, 00:43
If the exception is happening while decoding, it could be due to corrupt data. Avro allocates a List preallocated to the size encoded, and I've seen corrupted data cause attempted allocations of arrays too large for the heap.
On 6/9/11 4:58 PM, "Scott Carey" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: What is the stack trace on the out of memory exception? On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total. On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of t +
Scott Carey 2011-06-10, 00:43
-
RE: avro object reuseey-chih chow 2011-06-10, 17:53
Since this was in production, we did not turn on stack trace. Also, it was highly unlikely that there was any data corrupted because, if one mapper failed due to out of memory, the system started another one and went through all the data. From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 9 Jun 2011 17:43:02 -0700 Subject: Re: avro object reuse If the exception is happening while decoding, it could be due to corrupt data. Avro allocates a List preallocated to the size encoded, and I've seen corrupted data cause attempted allocations of arrays too large for the heap. On 6/9/11 4:58 PM, "Scott Carey" <[EMAIL PROTECTED]> wrote: What is the stack trace on the out of memory exception? On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total.On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are r +
ey-chih chow 2011-06-10, 17:53
-
Re: avro object reuseScott Carey 2011-06-10, 18:11
Corruption can occur in I/O busses and RAM. Does this tend to fail on the same nodes, or any node randomly? Since it does not fail consistently, this makes me suspect some sort of corruption even more.
I suggest turning on stack traces for fatal throwables. This shouldn't hurt production performance since they don't happen regularly and break the task anyway. Of the heap dumps seen so far, the primary consumption is byte[] and no more than 300MB. How large are your java heaps? On 6/10/11 10:53 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: Since this was in production, we did not turn on stack trace. Also, it was highly unlikely that there was any data corrupted because, if one mapper failed due to out of memory, the system started another one and went through all the data. ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 17:43:02 -0700 Subject: Re: avro object reuse If the exception is happening while decoding, it could be due to corrupt data. Avro allocates a List preallocated to the size encoded, and I've seen corrupted data cause attempted allocations of arrays too large for the heap. On 6/9/11 4:58 PM, "Scott Carey" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: What is the stack trace on the out of memory exception? On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total. On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus +
Scott Carey 2011-06-10, 18:11
-
RE: avro object reuseey-chih chow 2011-06-10, 19:07
We have many MR jobs running on production, but only one of them shows this kind of behavior. Is there any specific condition that corruption will occur? From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Fri, 10 Jun 2011 11:11:55 -0700 Subject: Re: avro object reuse Corruption can occur in I/O busses and RAM. Does this tend to fail on the same nodes, or any node randomly? Since it does not fail consistently, this makes me suspect some sort of corruption even more. I suggest turning on stack traces for fatal throwables. This shouldn't hurt production performance since they don't happen regularly and break the task anyway. Of the heap dumps seen so far, the primary consumption is byte[] and no more than 300MB. How large are your java heaps? On 6/10/11 10:53 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: Since this was in production, we did not turn on stack trace. Also, it was highly unlikely that there was any data corrupted because, if one mapper failed due to out of memory, the system started another one and went through all the data. From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 9 Jun 2011 17:43:02 -0700 Subject: Re: avro object reuse If the exception is happening while decoding, it could be due to corrupt data. Avro allocates a List preallocated to the size encoded, and I've seen corrupted data cause attempted allocations of arrays too large for the heap. On 6/9/11 4:58 PM, "Scott Carey" <[EMAIL PROTECTED]> wrote: What is the stack trace on the out of memory exception? On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total.On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih From: [EMAIL PROTECTED] To: [EMAIL PROTECTED] Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or per record? If this is called more than once per map job, it might explain this. In principle, Jackson is only used by a mapper during initialization. The below indicates that this may not be the case or that something outside of Avro is causing a lot of Jackson JSON parsing. Are you using something that is converting the Avro data to Json form? toString() on most Avro datum objects will do a lot of work with Jackson, for example — but the below are deserializer objects not serializer objects so that is not likely the issue. On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.H +
ey-chih chow 2011-06-10, 19:07
-
Re: avro object reuseScott Carey 2011-06-10, 22:58
It is just a hunch that an OOME can happen if a corrupted array size is read (since I have seen this before). Without the OOME stack trace, I can't say either way. Sometimes the OOME stack trace is useless, because other things leaked leading to it, and other times it can show the source of the problem because it happens during an attempt to allocate a very large object or object graph.
Because OOME is not of type Exception, but rather (Throwable/Error), it usually gets printed out somewhere (check the std err logs of the map job) even when logging is turned down. On 6/10/11 12:07 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We have many MR jobs running on production, but only one of them shows this kind of behavior. Is there any specific condition that corruption will occur? ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Fri, 10 Jun 2011 11:11:55 -0700 Subject: Re: avro object reuse Corruption can occur in I/O busses and RAM. Does this tend to fail on the same nodes, or any node randomly? Since it does not fail consistently, this makes me suspect some sort of corruption even more. I suggest turning on stack traces for fatal throwables. This shouldn't hurt production performance since they don't happen regularly and break the task anyway. Of the heap dumps seen so far, the primary consumption is byte[] and no more than 300MB. How large are your java heaps? On 6/10/11 10:53 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: Since this was in production, we did not turn on stack trace. Also, it was highly unlikely that there was any data corrupted because, if one mapper failed due to out of memory, the system started another one and went through all the data. ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 17:43:02 -0700 Subject: Re: avro object reuse If the exception is happening while decoding, it could be due to corrupt data. Avro allocates a List preallocated to the size encoded, and I've seen corrupted data cause attempted allocations of arrays too large for the heap. On 6/9/11 4:58 PM, "Scott Carey" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: What is the stack trace on the out of memory exception? On 6/9/11 4:45 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We configure more than 100MB for MapReduce to do sorting. Memory we allocate for doing other things in the mapper actually is larger, but, for this job, we always get out-of-meory exceptions and the job can not complete. We try to find out if there is a way to avoid this problem. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Thu, 9 Jun 2011 15:42:10 -0700 Subject: Re: avro object reuse The most likely candidate for creating many instances of BufferAccessor and ByteArrayByteSource is BinaryData.compare() and BinaryData.hashCode(). Each call will create one of each (hash) or two of each (compare). These are only 32 bytes per instance and quickly become garbage that is easily cleaned up by the GC. The below have only 32 bytes each and 8MB total. On the other hand, the byte[]'s appear to be about 24K each on average and are using 100MB. Is this the size of your configured MapReduce sort MB? On 6/9/11 3:08 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We did more monitoring. At one instance, we got the following histogram via Jmap. The question is why there are so many instances of BinaryDecoder$BufferAccessor and BinaryDecoder$ByteArrayByteSource. How to avoid this? Thanks. Object Histogram: num #instances #bytes Class description -----------------1: 4199 100241168 byte[] 2: 272948 8734336 org.apache.avro.io.BinaryDecoder$BufferAccessor 3: 272945 8734240 org.apache.avro.io.BinaryDecoder$ByteArrayByteSource 4: 2093 5387976 int[] 5: 23762 2822864 * ConstMethodKlass 6: 23762 1904760 * MethodKlass 7: 39295 1688992 * SymbolKlass 8: 2127 1216976 * ConstantPoolKlass 9: 2127 882760 * InstanceKlassKlass 10: 1847 742936 * ConstantPoolCacheKlass 11: 9602 715608 char[] 12: 1072 299584 * MethodDataKlass 13: 9698 232752 java.lang.String 14: 2317 222432 java.lang.Class 15: 3288 204440 short[] 16: 3167 156664 * System ObjArray 17: 2401 57624 java.util.HashMap$Entry 18: 666 53280 java.lang.reflect.Method 19: 161 52808 * ObjArrayKlassKlass 20: 1808 43392 java.util.Hashtable$Entry ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: RE: avro object reuse Date: Wed, 1 Jun 2011 15:14:03 -0700 We use a lot of toString() call on the avro Utf8 object. Will this cause Jackson call? Thanks. Ey-Chih ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Wed, 1 Jun 2011 13:38:39 -0700 Subject: Re: avro object reuse This is great info. Jackson should only be used once when the file is opened, so this is confusing from that point of view. Is something else using Jackson or initializing an Avro JsonDecoder frequently? There are over 100000 Jackson DeserializationConfig objects. Another place that parses the schema is in AvroSerialization.java. Does the Hadoop getDeserializer() API method get called once per job, or p +
Scott Carey 2011-06-10, 22:58
-
Re: avro object reuseScott Carey 2011-06-01, 20:45
Lower down this list of object counts, what are the top org.apache.avro.** object counts?
How many AvroSerialization objects? How many AvroMapper, AvroWrapper, etc? What about org.apache.hadoop.** objects? On 6/1/11 11:34 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We ran jmap on one of our mapper and found the top usage as follows: num #instances #bytes Class description -------------------------------------------------------------------------- 1: 24405 291733256 byte[] 2: 6056 40228984 int[] 3: 388799 19966776 char[] 4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser 5: 369623 11827936 java.lang.String 6: 111059 8769424 java.util.HashMap$Entry[] 7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext 8: 211374 6763968 java.util.HashMap$Entry 9: 102551 5742856 org.codehaus.jackson.util.TextBuffer 10: 105854 5080992 java.nio.HeapByteBuffer 11: 105821 5079408 java.nio.HeapCharBuffer 12: 104578 5019744 java.util.HashMap 13: 102551 4922448 org.codehaus.jackson.io.IOContext 14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig 15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer 16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext 17: 101779 4071160 java.io.StringReader 18: 101754 4070160 java.util.HashMap$KeyIterator It looks like Jackson eats up a lot of memory. Our mapper reads in files of the avro format. Does avro use Jackson a lot in reading the avro files? Is there any way to improve this? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Date: Tue, 31 May 2011 18:26:23 -0700 Subject: Re: avro object reuse All of those instances are short-lived. If you are running out of memory, its not likely due to object reuse. This tends to cause more CPU time in the garbage collector, but not out of memory conditions. This can be hard to do on a cluster, but grabbing 'jmap –histo' output from a JVM that has a larger-than-expected JVM heap usage can often be used to quickly identify the cause of memory consumption issues. I'm not sure if AvroUtf8InputFormat can safely re-use its instances of Utf8 or not. On 5/31/11 5:40 PM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: I actually looked into Avro code to find out how Avro does object reuse. I looked at AvroUtf8InputFormat and got the following question. Why a new Utf8 object has to be created each time the method next(AvroWrapper<Utf8> key, NullWritable value) is called ? Will this eat up too much memory when we call next(key, value) many times? Since Utf8 is mutable, can we just create one Utf8 object for all the calls to next(key, value)? Will this save memory? Thanks. Ey-Chih Chow ________________________________ From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> Subject: avro object reuse Date: Tue, 31 May 2011 10:38:39 -0700 Hi, We have several mapreduce jobs using avro. They take too much memory when running on production. Can anybody suggest some object reuse techniques to cut down memory usage? Thanks. Ey-Chih Chow +
Scott Carey 2011-06-01, 20:45
-
RE: avro object reuseey-chih chow 2011-06-01, 22:21
What follows is the whole output of our jmap. Hope this can help you identify the problem. num #instances #bytes Class description--------------------------------------------------------------------------1: 24405 291733256 byte[]2: 6056 40228984 int[]3: 388799 19966776 char[]4: 101779 16284640 org.codehaus.jackson.impl.ReaderBasedParser5: 369623 11827936 java.lang.String6: 111059 8769424 java.util.HashMap$Entry[]7: 204083 8163320 org.codehaus.jackson.impl.JsonReadContext8: 211374 6763968 java.util.HashMap$Entry9: 102551 5742856 org.codehaus.jackson.util.TextBuffer10: 105854 5080992 java.nio.HeapByteBuffer11: 105821 5079408 java.nio.HeapCharBuffer12: 104578 5019744 java.util.HashMap13: 102551 4922448 org.codehaus.jackson.io.IOContext14: 101782 4885536 org.codehaus.jackson.map.DeserializationConfig15: 101783 4071320 org.codehaus.jackson.sym.CharsToNameCanonicalizer16: 101779 4071160 org.codehaus.jackson.map.deser.StdDeserializationContext17: 101779 4071160 java.io.StringReader18: 101754 4070160 java.util.HashMap$KeyIterator19: 24001 3429704 * ConstMethodKlass20: 139087 3338088 java.lang.Long21: 115338 3215280 java.lang.Object[]22: 24001 2887768 * MethodKlass23: 2147 2414896 * ConstantPoolKlass24: 39532 2017320 * SymbolKlass25: 102735 1643760 java.util.HashMap$KeySet26: 2147 1596304 * InstanceKlassKlass27: 1865 1482184 * ConstantPoolCacheKlass28: 15780 1136160 com.sun.org.apache.xerces.internal.dom.DeferredElementNSImpl29: 27860 1114400 java.util.HashMap$EntryIterator30: 27585 1103400 com.sun.org.apache.xerces.internal.dom.DeferredTextImpl31: 1025 535536 * MethodDataKlass32: 5140 331816 short[]33: 5814 316424 java.lang.String[]34: 13135 315240 java.lang.StringBuilder35: 7723 247136 java.util.AbstractList$ListItr36: 1321 245632 org.apache.avro.io.parsing.Symbol[]37: 2332 242528 java.lang.Class38: 4712 226176 org.apache.avro.Schema$Props39: 6848 219136 java.util.AbstractList$Itr40: 12793 204688 java.lang.Integer41: 6033 193056 com.sun.org.apache.xerces.internal.xni.QName42: 4710 188400 java.util.LinkedHashMap$Entry43: 3190 171896 * System ObjArray44: 5228 167296 java.util.Hashtable$Entry45: 1789 114496 java.net.URL46: 777 100592 java.util.Hashtable$Entry[]47: 156 91104 * ObjArrayKlassKlass48: 3408 81792 java.util.ArrayList49: 450 64800 int[][]50: 90 64080 com.sun.org.apache.xerces.internal.util.SymbolTable$Entry[]51: 2513 60312 org.apache.avro.util.Utf852: 681 59928 java.lang.reflect.Method53: 1060 59360 java.util.LinkedHashMap54: 2160 51840 com.sun.org.apache.xerces.internal.util.XMLStringBuffer55: 1034 49632 org.apache.avro.Schema$Field56: 772 49408 org.codehaus.jackson.impl.WriterBasedGenerator57: 1980 47520 com.sun.org.apache.xerces.internal.xni.XMLString58: 775 43400 org.codehaus.jackson.map.ser.StdSerializerProvider59: 775 43400 org.codehaus.jackson.map.SerializationConfig60: 2596 41536 org.codehaus.jackson.node.TextNode61: 271 39128 java.lang.Object[][]62: 1564 37536 org.apache.avro.generic.GenericData$Record63: 900 36000 com.sun.org.apache.xerces.internal.xni.parser.XMLConfigurationException64: 360 34560 com.sun.org.apache.xerces.internal.xni.QName[]65: 720 34560 com.sun.org.apache.xerces.internal.util.XMLAttributesImpl$Attribute66: 1035 33120 java.util.LinkedHashMap$KeyIterator67: 2064 33024 java.util.HashMap$EntrySet68: 673 32304 java.util.Hashtable69: 90 30960 com.sun.org.apache.xerces.internal.impl.XMLNSDocumentScannerImpl70: 772 30880 org.codehaus.jackson.impl.ObjectWContext71: 949 30368 org.apache.avro.Schema$LockableArrayList72: 462 29568 java.util.regex.Matcher73: 1217 29208 java.lang.Double74: 900 28800 com.sun.org.apache.xerces.internal.util.AugmentationsImpl$SmallContainer75: 1077 25848 java.io.File76: 1035 24840 org.codehaus.jackson.node.ObjectNode77: 773 24736 org.codehaus.jackson.map.ser.ReadOnlyClassToSerializerMap78: 772 24704 org.codehaus.jackson.io.SegmentedStringWriter79: 772 24704 org.apache.avro.generic.GenericData$Array80: 772 24704 org.codehaus.jackson.impl.RootWContext81: 916 21984 org.apache.avro.Schema$ArraySchema82: 838 20112 org.apache.avro.Schema$StringSchema83: 620 19840 java.util.Vector84: 619 19808 org.apache.avro.io.parsing.Symbol$UnionAdjustAction85: 615 19680 com.sun.org.apache.xerces.internal.util.SymbolTable$Entry86: 180 18720 sun.net.www.protocol.file.FileURLConnection87: 776 18624 org.codehaus.jackson.map.ser.SerializerCache$UntypedKeyRaw88: 774 18576 org.apache.avro.Schema$UnionSchema89: 774 18576 org.codehaus.jackson.map.ser.SerializerCache$TypedKeyRaw90: 772 18528 org.apache.avro.mapred.Pair91: 772 18528 org.apache.avro.io.parsing.Symbol$Sequence92: 772 18528 org.apache.avro.Schema$SeenPair93: 770 18480 org.apache.avro.Schema$NullSchema94: 90 18000 com.sun.org.apache.xerces.internal.dom.DeferredDocumentImpl95: 544 17408 java.util.Stack96: 720 17280 com.sun.org.apache.xerces.internal.dom.DeferredDocumentImpl$RefCount97: 90 17280 com.sun.org.apache.xerces.internal.impl.XMLDTDScannerImpl98: 90 17280 com.sun.org.apache.xerces.internal.parsers.XIncludeAwareParserConfiguration99: 707 16968 org.codehaus.jackson.sym.CharsToNameCanonicalizer$Bucket100: 690 16560 org.codehaus.jackson.node.ArrayNode101: 754 16192 java.lang.Class[]102: 90 15840 com.sun.org.apache.xerces.internal.impl.dtd.XMLNSDTDValidator103: 90 15120 com.sun.org.apache.xerces.internal.xinclude.XIncludeHandler104: 605 14520 java.lang.StringBuffer105: 389 14472 boolean[]106: 450 14400 com.sun.org.apache.xerces.internal.util.XMLResourceIdentifierImpl107: 900 14400 com.sun.org.apache.xerces.internal.util.AugmentationsImpl108: 570 13680 java.net.URLClassLoader$2109: 184 13248 java.lang.reflect.Field110: 92 13248 org.codehaus.jackson.sym.CharsToNameCanonicalizer$Bucket[]111: 90 12960 com.sun.org.apache.xerces.internal.parsers.DOMParser112: 773 12368 org.codehaus.jackson.map.ser.SerializerCache$TypedKeyFull113: 171 12312 java.lang.reflect.Constructor114: 307 12280 java.lang.re +
ey-chih chow 2011-06-01, 22:21
-
Re: avro object reuseTatu Saloranta 2011-06-02, 00:13
On Wed, Jun 1, 2011 at 1:45 PM, Scott Carey <[EMAIL PROTECTED]> wrote:
> Lower down this list of object counts, what are the top org.apache.avro.** > object counts? > How many AvroSerialization objects? How many AvroMapper, AvroWrapper, etc? > What about org.apache.hadoop.** objects? Also: is this jmap view of live objects, or just dump of ALL objects, live and dead? It seems like dump of latter, as most Jackson objects are short-term things created for per-invocation purposes, and discarded after process is complete. High count is not necessarily surprising for high-throughput systems; it is only odd if these are actual live objects. -+ Tatu +- +
Tatu Saloranta 2011-06-02, 00:13
-
Re: avro object reuseScott Carey 2011-06-02, 00:45
It would be useful to get a 'jmap -histo:live' report as well, which will
only have items that remain after a full GC. However, a high churn of short lived Jackson objects is not expected here unless the user is reading Json serialized files and not Avro binary. Avro Data Files only contain binary encoded Avro content. It would be surprising to see many Jackson objects here if reading Avro Data Files, because we expect to use Jackson to parse an Avro schema from json only once or twice per file. After the schema is parsed, Jackson shouldn't be used. A hundred thousand DeserializationConfig instances means that isn't the case. On 6/1/11 5:13 PM, "Tatu Saloranta" <[EMAIL PROTECTED]> wrote: >On Wed, Jun 1, 2011 at 1:45 PM, Scott Carey <[EMAIL PROTECTED]> >wrote: >> Lower down this list of object counts, what are the top >>org.apache.avro.** >> object counts? >> How many AvroSerialization objects? How many AvroMapper, AvroWrapper, >>etc? >> What about org.apache.hadoop.** objects? > >Also: is this jmap view of live objects, or just dump of ALL objects, >live and dead? >It seems like dump of latter, as most Jackson objects are short-term >things created for per-invocation purposes, and discarded after >process is complete. High count is not necessarily surprising for >high-throughput systems; it is only odd if these are actual live >objects. > >-+ Tatu +- +
Scott Carey 2011-06-02, 00:45
-
Re: avro object reuseTatu Saloranta 2011-06-02, 00:48
On Wed, Jun 1, 2011 at 5:45 PM, Scott Carey <[EMAIL PROTECTED]> wrote:
> It would be useful to get a 'jmap -histo:live' report as well, which will > only have items that remain after a full GC. > > However, a high churn of short lived Jackson objects is not expected here > unless the user is reading Json serialized files and not Avro binary. > Avro Data Files only contain binary encoded Avro content. > > It would be surprising to see many Jackson objects here if reading Avro > Data Files, because we expect to use Jackson to parse an Avro schema from > json only once or twice per file. After the schema is parsed, Jackson > shouldn't be used. A hundred thousand DeserializationConfig instances > means that isn't the case. Right -- it indicates that something (else) is using Jackson; and there will typically be one instance of DeserializationConfig for each data-binding call (ObjectMapper.readValue()), as a read-only copy is made for operation. ... or if something is reading schema that many times, that sounds like a problem in itself. -+ Tatu +- +
Tatu Saloranta 2011-06-02, 00:48
-
Re: avro object reuseScott Carey 2011-06-02, 01:48
One thing we do right now that might be related is the following:
We keep Avro default Schema values as JsonNode objects. While traversing the JSON Avro schema representation using ObjectMapper.readTree() we remember JsonNodes that are "default" properties on fields and keep them on the Schema object. If these keep references to the parent (and the whole JSON tree, or worse, the ObjectMapper and input stream) it would be poor use of Jackson by us; although we'd need a way to keep a detached JsonNode or equivalent. However, even if that is the case (which it does not seem to be -- the jmap output has no JsonNode instances), it doesn't explain why we would be calling ObjectMapper frequently. We only call ObjectMapper.readTree(JsonParser) when creating a Schema from JSON. We call JsonNode methods from extracted fragments for everything else. This brings me to the following suspicion based on the data: Somewhere, Schema objects are being created frequently via one of the Schema.parse() or Protocol.parse() static methods. On 6/1/11 5:48 PM, "Tatu Saloranta" <[EMAIL PROTECTED]> wrote: >On Wed, Jun 1, 2011 at 5:45 PM, Scott Carey <[EMAIL PROTECTED]> >wrote: >> It would be useful to get a 'jmap -histo:live' report as well, which >>will >> only have items that remain after a full GC. >> >> However, a high churn of short lived Jackson objects is not expected >>here >> unless the user is reading Json serialized files and not Avro binary. >> Avro Data Files only contain binary encoded Avro content. >> >> It would be surprising to see many Jackson objects here if reading Avro >> Data Files, because we expect to use Jackson to parse an Avro schema >>from >> json only once or twice per file. After the schema is parsed, Jackson >> shouldn't be used. A hundred thousand DeserializationConfig instances >> means that isn't the case. > >Right -- it indicates that something (else) is using Jackson; and >there will typically be one instance of DeserializationConfig for each >data-binding call (ObjectMapper.readValue()), as a read-only copy is >made for operation. >... or if something is reading schema that many times, that sounds >like a problem in itself. > >-+ Tatu +- +
Scott Carey 2011-06-02, 01:48
-
RE: avro object reuseey-chih chow 2011-06-02, 17:23
We create GenericData.Record a lot in our code via new GenericData.Record(schema). Will this generates Jackson calls? Thanks. Ey-Chih Chow > From: [EMAIL PROTECTED] > To: [EMAIL PROTECTED] > Date: Wed, 1 Jun 2011 18:48:15 -0700 > Subject: Re: avro object reuse > > One thing we do right now that might be related is the following: > > We keep Avro default Schema values as JsonNode objects. While traversing > the JSON Avro schema representation using ObjectMapper.readTree() we > remember JsonNodes that are "default" properties on fields and keep them > on the Schema object. > If these keep references to the parent (and the whole JSON tree, or worse, > the ObjectMapper and input stream) it would be poor use of Jackson by us; > although we'd need a way to keep a detached JsonNode or equivalent. > > However, even if that is the case (which it does not seem to be -- the > jmap output has no JsonNode instances), it doesn't explain why we would be > calling ObjectMapper frequently. We only call > ObjectMapper.readTree(JsonParser) when creating a Schema from JSON. We > call JsonNode methods from extracted fragments for everything else. > > > This brings me to the following suspicion based on the data: > Somewhere, Schema objects are being created frequently via one of the > Schema.parse() or Protocol.parse() static methods. > > On 6/1/11 5:48 PM, "Tatu Saloranta" <[EMAIL PROTECTED]> wrote: > > >On Wed, Jun 1, 2011 at 5:45 PM, Scott Carey <[EMAIL PROTECTED]> > >wrote: > >> It would be useful to get a 'jmap -histo:live' report as well, which > >>will > >> only have items that remain after a full GC. > >> > >> However, a high churn of short lived Jackson objects is not expected > >>here > >> unless the user is reading Json serialized files and not Avro binary. > >> Avro Data Files only contain binary encoded Avro content. > >> > >> It would be surprising to see many Jackson objects here if reading Avro > >> Data Files, because we expect to use Jackson to parse an Avro schema > >>from > >> json only once or twice per file. After the schema is parsed, Jackson > >> shouldn't be used. A hundred thousand DeserializationConfig instances > >> means that isn't the case. > > > >Right -- it indicates that something (else) is using Jackson; and > >there will typically be one instance of DeserializationConfig for each > >data-binding call (ObjectMapper.readValue()), as a read-only copy is > >made for operation. > >... or if something is reading schema that many times, that sounds > >like a problem in itself. > > > >-+ Tatu +- > +
ey-chih chow 2011-06-02, 17:23
-
Re: avro object reuseScott Carey 2011-06-02, 21:14
No, that should not trigger Jackson parsing. Schema.parse() and Protocol.parse() do.
On 6/2/11 10:23 AM, "ey-chih chow" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: We create GenericData.Record a lot in our code via new GenericData.Record(schema). Will this generates Jackson calls? Thanks. Ey-Chih Chow > From: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> > To: [EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]> > Date: Wed, 1 Jun 2011 18:48:15 -0700 > Subject: Re: avro object reuse > > One thing we do right now that might be related is the following: > > We keep Avro default Schema values as JsonNode objects. While traversing > the JSON Avro schema representation using ObjectMapper.readTree() we > remember JsonNodes that are "default" properties on fields and keep them > on the Schema object. > If these keep references to the parent (and the whole JSON tree, or worse, > the ObjectMapper and input stream) it would be poor use of Jackson by us; > although we'd need a way to keep a detached JsonNode or equivalent. > > However, even if that is the case (which it does not seem to be -- the > jmap output has no JsonNode instances), it doesn't explain why we would be > calling ObjectMapper frequently. We only call > ObjectMapper.readTree(JsonParser) when creating a Schema from JSON. We > call JsonNode methods from extracted fragments for everything else. > > > This brings me to the following suspicion based on the data: > Somewhere, Schema objects are being created frequently via one of the > Schema.parse() or Protocol.parse() static methods. > > On 6/1/11 5:48 PM, "Tatu Saloranta" <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> wrote: > > >On Wed, Jun 1, 2011 at 5:45 PM, Scott Carey <[EMAIL PROTECTED]<mailto:[EMAIL PROTECTED]>> > >wrote: > >> It would be useful to get a 'jmap -histo:live' report as well, which > >>will > >> only have items that remain after a full GC. > >> > >> However, a high churn of short lived Jackson objects is not expected > >>here > >> unless the user is reading Json serialized files and not Avro binary. > >> Avro Data Files only contain binary encoded Avro content. > >> > >> It would be surprising to see many Jackson objects here if reading Avro > >> Data Files, because we expect to use Jackson to parse an Avro schema > >>from > >> json only once or twice per file. After the schema is parsed, Jackson > >> shouldn't be used. A hundred thousand DeserializationConfig instances > >> means that isn't the case. > > > >Right -- it indicates that something (else) is using Jackson; and > >there will typically be one instance of DeserializationConfig for each > >data-binding call (ObjectMapper.readValue()), as a read-only copy is > >made for operation. > >... or if something is reading schema that many times, that sounds > >like a problem in itself. > > > >-+ Tatu +- > +
Scott Carey 2011-06-02, 21:14
|