Hi Asim,

Here's Wellington's answer:

"""
This jute buffer len error generally means a given znode being watched/read
had grown too large to fit into the buffer. It's not specific to number of
watches attached, but amount of info stored in it, for example, too many
children znode under a given znode. In order to understand what's behind
the error, you should analyse your zookeeper znodes tree, you may have a
hint by looking at zookeeper snapashot files. Would you have replication
enabled on this cluster? A common cause for such errors in hbase is when
replication is slow/stuck, and source cluster is under heavy write load,
causing replication queue to grow much faster than it's ability to drain,
which will imply on many znodes created under "replication" znode.
"""

It's a great answer, hope it helps.

Andor



On Tue, Mar 12, 2019 at 2:33 AM Asim Zafir <asim.za...@gmail.com> wrote:

> Thank. I haven’t seen any response from  Wellington Chevreuil yet. Quite
> strange, if you can forward me the email, please do.
>
> Region server exits were primarily due to ZK loosing due to len ERROR
> increasing 32 MB thrashed. That is a ZK settings. So far upon my research
> jute-buffer is related to the number of watchers that is set on ephemeral
> node in zk. the more then number of watchers on the node the more data will
> be serialized by the jute library and that’s where jute.buffer cap is
> hitting its mark. either the number of watchers have increased or perhaps
> made redundant on a given znode/epheremal node but once again HBase knows
> better how it is using the ZK and how can it impact the jute-buffer. i
> would love to see what HBase community has responded.
>
>
> > On Mar 12, 2019, at 2:26 AM, Justin Ling Mao <maoling199210...@sina.com>
> wrote:
> >
> > - I also saw that you posted your question to the hbase community, Are
> you happy with the answer from Wellington Chevreuil? are you creating too
> many zk child nodes?
> > - "excessive regions server" is the root cause? what you should do is
> calming this symptoms of hbase,other than increasing the jute.maxbuffer to
> a large value(128MB)?
> > --->"Upon increase the jute buffer we are saying significant stability"
> Your hbase may be stable temporary, but zk will be panic.the logic is
> here:when write/get too large data to/from zk,it will give too much
> pressure on the server, sometimes cause the GC stop in the leader, follwers
> cannot ping the leader within the bounded time, then start to re-election.
> I guess,haha:D
> >
> > ----- Original Message -----
> > From: Asim Zafir <asim.za...@gmail.com>
> > To: maoling199210...@sina.com
> > Cc: dev <dev@zookeeper.apache.org>
> > Subject: Re: Re: Jute buffer size increasing.
> > Date: 2019-03-11 21:14
> >
> > Thanks for the response. Actually it was otherwise - when he had the
> buffer
> > size set to 32 mb, we were getting Len ERRORS reported by ZK after which
> ZK
> > was tearing down the connection. We saw excessive regions server exist
> and
> > at times complete HBase crash. It was very painful. Upon increase the
> jute
> > buffer we are saying significant stability. Can you explain he logic
> behind
> > jute buffer and instability that you were seeing, I was not able to find
> a
> > good resources to relate that level of detail so it will be extremely
> > helpful if you can explain.
> > On Sun, Mar 10, 2019 at 7:26 PM Justin Ling Mao <
> maoling199210...@sina.com>
> > wrote:
> > > - What's your user case using zk for hbase? Why increase
> jute.maxbuffer to
> > > 128MB? incredible! really need this change?
> > > - if the jute.maxbuffer is too large, it will have a side effect on the
> > > stability and throughout of the cluster,the response time between
> server
> > > and client.
> > > - for the "jmap -histo:live" you had provided, you have created almost
> > > 103842 zk-nodes,and their memory usage is less than 10 MB, so don't
> worry
> > > about this.
> > >
> > > ----- Original Message -----
> > > From: Asim Zafir <asim.za...@gmail.com>
> > > To: Jordan Zimmerman <jor...@jordanzimmerman.com>
> > > Cc: Zookeeper <dev@zookeeper.apache.org>
> > > Subject: Re: Jute buffer size increasing.
> > > Date: 2019-03-09 10:11
> > >
> > > thanks and very much appreciate your feedback.
> > > here the system that is using zookeeper is HBASE and you are right its
> a
> > > manual setting. we had it 1mb, then increase to 20mb, 32mb and now 128
> mb..
> > > what is the impact of having higher jute buffer size and more (higher
> count
> > > of org.apache.zookeeper.data.StatPersisted object).
> > > On Fri, Mar 8, 2019 at 5:24 PM Jordan Zimmerman <
> > > jor...@jordanzimmerman.com>
> > > wrote:
> > > > The Jute buffer size is a manual setting, so it wouldn't increase by
> > > > itself. If you find that you have to keep increasing it's due to a
> few
> > > > possibilities: ZNodes with lots of children (with long names). Any 1
> > > > ZooKeeper API call is limited by jute max buffer. So, if a call to
> > > > getChildren() where there's a lot of children or lots of children
> with
> > > long
> > > > names (or both) can bust jute max buffer. Another possibility is
> ZNodes
> > > > with large payloads.
> > > >
> > > > -JZ
> > > >
> > > > On Mar 8, 2019, at 7:21 PM, Asim Zafir <asim.za...@gmail.com> wrote:
> > > >
> > > >
> > > > + ZK dev community. Please assist.
> > > > On Fri, Mar 8, 2019 at 4:10 PM Asim Zafir <asim.za...@gmail.com>
> wrote:
> > > >
> > > >> Hi Jordon,
> > > >>
> > > >> We are seeing constance increase in jute buffer size on our
> zookeeper
> > > >> instance. right now it is set to 128. We are primarily using
> zookeeper
> > > for
> > > >> HBase cluster. I want to see what is contributing to the increase of
> > > jute
> > > >> buffer size but so for after investigating the code and studying the
> > > >> protocol itself it appear it is a function of number of watches that
> > > gets
> > > >> set on the znodes. to see how many zookeeper watch objects are on
> > > zookeeper
> > > >> jvm /instance I did a jmap history:live on zookeeper pid and I got
> the
> > > >> following output (please see below). I am not sure what is [C, [B
> here
> > > and
> > > >> it doesn't appear its refers to any class - I don't see this on dev
> > > >> instance of zookeeper. due to suspect memory leak or another issue?
> > > Please
> > > >> guide me through this as I can't find a resource who can go that
> far to
> > > >> give me any hint as to what may be happening on my end. Also is it
> safe
> > > for
> > > >> ZK sizes to increase that much? I will greatly appreciate your
> feedback
> > > and
> > > >> help on this.
> > > >>
> > > >> num #instances #bytes class name
> > > >> ----------------------------------------------
> > > >> 1: 220810 140582448 [C
> > > >> 2: 109370 34857168 [B
> > > >> 3: 103842 7476624
> > > >> org.apache.zookeeper.data.StatPersisted
> > > >> 4: 220703 5296872 java.lang.String
> > > >> 5: 28682 3783712 <constMethodKlass>
> > > >> 6: 28682 3681168 <methodKlass>
> > > >> 7: 111000 3552000 java.util.HashMap$Entry
> > > >> 8: 107569 3442208
> > > >> java.util.concurrent.ConcurrentHashMap$HashEntry
> > > >> 9: 103842 3322944 org.apache.zookeeper.server.DataNode
> > > >> 10: 2655 3179640 <constantPoolKlass>
> > > >> 11: 2313 2017056 <constantPoolCacheKlass>
> > > >> 12: 2655 1842456 <instanceKlassKlass>
> > > >> 13: 318 1241568
> > > >> [Ljava.util.concurrent.ConcurrentHashMap$HashEntry;
> > > >> 14: 7526 1221504 [Ljava.util.HashMap$Entry;
> > > >> 15: 1820 812976 <methodDataKlass>
> > > >> 16: 8228 394944 java.util.HashMap
> > > >> 17: 2903 348432 java.lang.Class
> > > >> 18: 4077 229688 [S
> > > >> 19: 4138 221848 [[I
> > > >> 20: 231 125664 <objArrayKlassKlass>
> > > >> 21: 7796 124736 java.util.HashSet
> > > >> 22: 6771 108336 java.util.HashMap$KeySet
> > > >> 23: 1263 62968 [Ljava.lang.Object;
> > > >> 24: 746 59680 java.lang.reflect.Method
> > > >> 25: 3570 57120 java.lang.Object
> > > >> 26: 502 36144 org.apache.zookeeper.server.Request
> > > >> 27: 649 25960 java.lang.ref.SoftReference
> > > >> 28: 501 24048 org.apache.zookeeper.txn.TxnHeader
> > > >> 29: 188 21704 [I
> > > >> 30: 861 20664 java.lang.Long
> > > >> 31: 276 19872 java.lang.reflect.Constructor
> > > >> 32: 559 17888
> > > >> java.util.concurrent.locks.ReentrantLock$NonfairSync
> > > >> 33: 422 16880 java.util.LinkedHashMap$Entry
> > > >> 34: 502 16064
> > > >> org.apache.zookeeper.server.quorum.QuorumPacket
> > > >> 35: 455 14560 java.util.Hashtable$Entry
> > > >> 36: 495 14368 [Ljava.lang.String;
> > > >> 37: 318 12720
> > > >> java.util.concurrent.ConcurrentHashMap$Segment
> > > >> 38: 3 12336 [Ljava.nio.ByteBuffer;
> > > >> 39: 514 12336 javax.management.ObjectName$Property
> > > >> 40: 505 12120 java.util.LinkedList$Node
> > > >> 41: 501 12024
> > > >> org.apache.zookeeper.server.quorum.Leader$Proposal
> > > >> 42: 619 11920 [Ljava.lang.Class;
> > > >> 43: 74 11840
> > > >> org.apache.zookeeper.server.NIOServerCnxn
> > > >> 44: 145 11672 [Ljava.util.Hashtable$Entry;
> > > >> 45: 729 11664 java.lang.Integer
> > > >> 46: 346 11072 java.lang.ref.WeakReference
> > > >> 47: 449 10776 org.apache.zookeeper.txn.SetDataTxn
> > > >> 48: 156 9984
> > > >> com.cloudera.cmf.event.shaded.org.apache.avro.Schema$Props
> > > >> 49: 266 8512 java.util.Vector
> > > >> 50: 75 8400 sun.nio.ch.SocketChannelImpl
> > > >> 51: 175 8400 java.nio.HeapByteBuffer
> > > >> 52: 247 8320
> > > >> [Ljavax.management.ObjectName$Property;
> > > >> 53: 303 7272 com.cloudera.cmf.event.EventCode
> > > >> 54: 300 7200 java.util.ArrayList
> > > >> 55: 136 6528 java.util.Hashtable
> > > >> 56: 156 6240 java.util.WeakHashMap$Entry
> > > >> 57: 194 6208
> > > >> com.sun.jmx.mbeanserver.ConvertingMethod
> > > >>
> > > >
> > > >
> > >
>
>

Reply via email to