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 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
