You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@zookeeper.apache.org by Aaron Crow <di...@yahoo.com> on 2010/05/13 05:30:48 UTC

Re: Pathological ZK cluster: 1 server verbosely WARN'ing, other 2 servers pegging CPU

I may have a better idea of what caused the trouble. I way, WAY
underestimated the number of nodes we collect over time. Right now we're at
1.9 million. This isn't a bug of our application; it's actually a feature
(but perhaps an ill-conceived one).

A most recent snapshot from a Zookeeper db is 227MB. If I scp it over to one
of the other Zookeeper hosts, it takes about 4 seconds.

Now, there are some things I can do to limit the number of nodes we collect.
My question is, how deadly could this node size be for us? Patrick mentioned
to me that he's run Zookeeper with this many nodes, but you need to be
careful about tuning. We're currently running with the recommended JVM
settings (see below). We're using different drives for the 2 different kinds
of data dirs that Zookeeper needs. We may also have the option of running on
a 64 bit OS with added RAM, if it's worth it. What about timeout settings?
I'm copying in our current settings below, are those ok?

Or should we just figure out how to keep our node count much lower? And how
low is "definitely pretty safe"?


=== some current settings ===
-XX:ParallelGCThreads=8 -XX:+UseConcMarkSweepGC -Xms2560m -Xmx2560m
tickTime=2000
initLimit=10
syncLimit=5
====================


Many thanks in advance for any good advice.
Aaron


On Wed, Apr 28, 2010 at 10:47 PM, Patrick Hunt <ph...@apache.org> wrote:

> Hi Aaron, some questions/comments below:
>
>
> On 04/28/2010 06:29 PM, Aaron Crow wrote:
>
>> We were running version 3.2.2 for about a month and it was working well
>> for
>> us. Then late this past Saturday night, our cluster went pathological. One
>> of the 3 ZK servers spewed many WARNs (see below), and the other 2 servers
>> were almost constantly pegging the CPU. All three servers are on separate
>> machines. From what we could tell, the machines were fine... networking
>> fine, disk fine, etc. The ZK clients were completely unable to complete
>> their connections to ZK.
>>
>>
> These machines are local (not wan) connected then? What OS and java version
> are you using?
>
> Do you see any FATAL or ERROR level messages in the logs?
>
> It would help to look at your zk config files for these servers. Could you
> provide (you might want to create a JIRA first, then just attach configs and
> other details/collateral to that, easier than dealing with email)
>
> If you have logs for the time period and can share that would be most
> useful. (again, gzip and attach to the jira)
>
>
>  We tried all sorts of restarts, running zkCleanup, etc. We even completely
>> shut down our clients... and the pathology continued. Our workaround was
>> to
>> do an urgent upgrade to version 3.3.0. The new ZK cluster with 3.3.0 has
>> been running well for us... so far...
>>
>>
> Off hand and with the data we have so far nothing sticks out that 3.3 would
> have resolved (JIRA is conveniently down for the last hour or so so I can't
> review right now). Although there were some changes to reduce memory
> consumption (see below).
>
>
>  I realize that, sadly, this message doesn't contain nearly enough details
>> to
>> trace exactly what happened. I guess I'm wondering if anyone has seen this
>> general scenario, and/or knows how to prevent? Is there anything we might
>> be
>> doing client side to trigger this? Our application level request frequency
>> is maybe a few requests to Zookeeper per second, times 5 clients
>> applications. If we detect a SESSION EXPIRED, we do a simple "create new
>> client and use that instead". And we were seeing this happen occasionally.
>>
>>
> What are the client doing? Do you have a large number/size of znodes?
>
> Do you see any OutOfMemoryError in the logs?
>
> Could the ZK server java process be swapping? Are you monitoring GC,
> perhaps large GC pauses are happening?
>
> I have a suspicion that one of a few things might be happening. I see the
> following in your original email:
>
>
> > :FollowerHandler@302] - Sending snapshot last zxid of peer is
> 0xd0007d66d
> > zxid of leader is 0xf00000000
> > 2010-04-24 23:06:03,254 - ERROR [FollowerHandler-/10.0.10.116:34405
> > :FollowerHandler@415] - Unexpected exception causing shutdown while sock
> > still open
> > java.net.SocketException: Broken pipe
> >          at java.net.SocketOutputStream.socketWrite0(Native Method)
> >          at
>
> ^^^^ this log section is saying that a ZK server (follower) connected to
> the elected leader and started to download (leader is sending here) the
> latest ZK snapshot (db) to the follower while it is recovering. However
> before the follower fully downloads the snapshot the connection (pipe) is
> broken and this operation fails. At this point the leader itself will
> probably give up the lead, a new election will happen, and it might be the
> case that this same problem occurs - the follower tries to download the
> snapshot but it again fails (and this loops forever)
>
> This issue can happen if the servers are slowed, either by slow
> interconnects (wan), network connectivity/perf problems, swapping of the
> JVM, and most commonly GC in the VM.
>
> See this recent case:
> http://www.mail-archive.com/zookeeper-dev@hadoop.apache.org/msg08083.html
>
> What is the size of your snapshot file? (you can see this in your zookeeper
> datadirectory, unless you wiped this during your attempt to
> cleanup/recover/upgrade?).
>
> Seeing your config file, and getting a sense of the snapshot file size will
> give us more insight.
>
> Patrick
>
>
>  Many, many thanks in advance for any insights or advice.
>>
>>
>> Example log output from the ZK server that spewed many WARNs:
>>
>> ===
>>
>> 2010-04-24 22:31:24,551 - WARN
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@318] - Exception when
>> following
>> the leader
>> java.net.SocketTimeoutException: Read timed out
>> at java.net.SocketInputStream.socketRead0(Native Method)
>> at java.net.SocketInputStream.read(SocketInputStream.java:129)
>> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
>> at java.io.DataInputStream.readInt(DataInputStream.java:370)
>> at org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
>> at
>>
>> org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:66)
>> at
>> org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:108)
>> at
>> org.apache.zookeeper.server.quorum.Follower.readPacket(Follower.java:114)
>> at
>>
>> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:193)
>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:525)
>> 2010-04-24 22:31:24,552 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@436] - shutdown called
>> java.lang.Exception: shutdown Follower
>> at org.apache.zookeeper.server.quorum.Follower.shutdown(Follower.java:436)
>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:529)
>> 2010-04-24 22:31:24,552 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:QuorumPeer@514] - LOOKING
>>
>> ...
>>
>> 2010-04-24 22:31:27,257 - WARN
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>> - Exception causing close of session 0x0 due to java.io.IOException:
>> ZooKeeperServer not running
>> 2010-04-24 22:31:27,257 - INFO
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>> - closing session:0x0 NIOServerCnxn:
>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>> 127.0.0.1:44589]
>> 2010-04-24 22:31:27,705 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:FastLeaderElection@706] -
>> Notification:
>> 3, 60129542145, 2, 1, LOOKING, LEADING, 3
>> 2010-04-24 22:31:27,705 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:QuorumPeer@523] - FOLLOWING
>> 2010-04-24 22:31:27,705 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:ZooKeeperServer@160] - Created server
>> 2010-04-24 22:31:27,706 - INFO
>>  [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@147] - Following sum08/
>> 10.0.10.118:2888
>> 2010-04-24 22:31:29,040 - WARN
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>> - Exception causing close of session 0x0 due to java.io.IOException:
>> ZooKeeperServer not running
>> 2010-04-24 22:31:29,040 - INFO
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>> - closing session:0x0 NIOServerCnxn:
>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>> 127.0.0.1:44591]
>> 2010-04-24 22:31:31,069 - WARN
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>> - Exception causing close of session 0x0 due to java.io.IOException:
>> ZooKeeperServer not running
>> 2010-04-24 22:31:31,069 - INFO
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>> - closing session:0x0 NIOServerCnxn:
>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>> 127.0.0.1:44592]
>> 2010-04-24 22:31:32,589 - WARN
>>  [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>> - Exception causing close of session 0x0 due to java.io.IOException:
>> ZooKeeperServer not running
>>
>> ===
>>
>> Example log output from a server that was pegging CPU:
>>
>> 010-04-24 23:05:27,911 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>> 2010-04-24 23:05:27,912 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>> 2010-04-24 23:05:27,910 - WARN  [Thread-15:QuorumCnxManager$SendWorker@539
>> ]
>> - Interrupted while waiting for message on queue
>> java.lang.InterruptedException
>>         at
>>
>> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:1899)
>>         at
>>
>> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1934)
>>         at
>> java.util.concurrent.ArrayBlockingQueue.take(ArrayBlockingQueue.java:317)
>>         at
>>
>> org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:533)
>> 2010-04-24 23:05:27,912 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>> 2010-04-24 23:05:27,912 - WARN  [Thread-16:QuorumCnxManager$RecvWorker@623
>> ]
>> - Connection broken:
>> java.nio.channels.AsynchronousCloseException
>>         at
>>
>> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:185)
>>         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:263)
>>         at
>>
>> org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:594)
>> 2010-04-24 23:05:27,912 - WARN  [Thread-15:QuorumCnxManager$SendWorker@554
>> ]
>> - Send worker leaving thread
>>
>> ...
>>
>> 010-04-24 23:05:51,952 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>> 2010-04-24 23:05:57,148 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>> 2010-04-24 23:06:03,243 - INFO  [FollowerHandler-/10.0.10.116:34405
>> :FollowerHandler@227] - Follower sid: 1 : info :
>> org.apache.zookeeper.server.quorum.QuorumPeer$QuorumServer@364641
>> 2010-04-24 23:06:03,244 - WARN  [FollowerHandler-/10.0.10.116:34405
>> :FollowerHandler@302] - Sending snapshot last zxid of peer is 0xd0007d66d
>> zxid of leader is 0xf00000000
>> 2010-04-24 23:06:03,254 - ERROR [FollowerHandler-/10.0.10.116:34405
>> :FollowerHandler@415] - Unexpected exception causing shutdown while sock
>> still open
>> java.net.SocketException: Broken pipe
>>         at java.net.SocketOutputStream.socketWrite0(Native Method)
>>         at
>> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
>>         at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
>>         at
>> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
>>         at
>> java.io.BufferedOutputStream.write(BufferedOutputStream.java:109)
>>         at java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
>>         at
>> org.apache.jute.BinaryOutputArchive.writeLong(BinaryOutputArchive.java:59)
>>         at
>> org.apache.zookeeper.server.DataNode.serialize(DataNode.java:129)
>>         at
>>
>> org.apache.jute.BinaryOutputArchive.writeRecord(BinaryOutputArchive.java:123)
>>         at
>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:917)
>>         at
>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>         at
>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>         at
>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>         at
>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>         at
>> org.apache.zookeeper.server.DataTree.serialize(DataTree.java:979)
>>         at
>>
>> org.apache.zookeeper.server.util.SerializeUtils.serializeSnapshot(SerializeUtils.java:104)
>>         at
>>
>> org.apache.zookeeper.server.ZooKeeperServer.serializeSnapshot(ZooKeeperServer.java:272)
>>         at
>>
>> org.apache.zookeeper.server.quorum.FollowerHandler.run(FollowerHandler.java:307)
>> 2010-04-24 23:06:03,255 - WARN  [FollowerHandler-/10.0.10.116:34405
>> :FollowerHandler@421] - ******* GOODBYE /10.0.10.116:34405 ********
>> 2010-04-24 23:06:03,550 - INFO  [WorkerReceiver
>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>> notification.
>>
>>

Re: Pathological ZK cluster: 1 server verbosely WARN'ing, other 2 servers pegging CPU

Posted by Ted Dunning <te...@gmail.com>.
Yes.  That is roughly what I mean.

If one server starts a GC, it can effectively go offline.  That might
pressure the other servers enough that one of them starts a GC.

This is unlikely with your GC settings, but you should turn on the verbose
GC logging to be sure.

On Wed, May 12, 2010 at 10:09 PM, Aaron Crow <di...@yahoo.com>wrote:

> But, sorry, I'm not sure what you
> mean about synchronized GC? You mean maybe all our Zookeepers get hit by a
> big GC at the same time, due to the very large db?
>

Re: Pathological ZK cluster: 1 server verbosely WARN'ing, other 2 servers pegging CPU

Posted by Aaron Crow <di...@yahoo.com>.
Hi Ted, yeah it's a big number, eh? We're essentially using Zookeeper to
track the state of cache entries, and currently we don't bound our cache. I
didn't realize how many entries we grow to over a long period of time, until
I started counting nodes in Zookeeper. But, sorry, I'm not sure what you
mean about synchronized GC? You mean maybe all our Zookeepers get hit by a
big GC at the same time, due to the very large db?


On Wed, May 12, 2010 at 9:21 PM, Ted Dunning <te...@gmail.com> wrote:

> Impressive number here, especially at your quoted "few per second" rate.
>
> Are you sure that you haven't inadvertently synchronized GC on multiple
> machines?
>
> On Wed, May 12, 2010 at 8:30 PM, Aaron Crow <di...@yahoo.com>
> wrote:
>
> > Right now we're at
> > 1.9 million. This isn't a bug of our application; it's actually a feature
> > (but perhaps an ill-conceived one).
> >
>

Re: Pathological ZK cluster: 1 server verbosely WARN'ing, other 2 servers pegging CPU

Posted by Ted Dunning <te...@gmail.com>.
Impressive number here, especially at your quoted "few per second" rate.

Are you sure that you haven't inadvertently synchronized GC on multiple
machines?

On Wed, May 12, 2010 at 8:30 PM, Aaron Crow <di...@yahoo.com> wrote:

> Right now we're at
> 1.9 million. This isn't a bug of our application; it's actually a feature
> (but perhaps an ill-conceived one).
>

Re: Pathological ZK cluster: 1 server verbosely WARN'ing, other 2 servers pegging CPU

Posted by Patrick Hunt <ph...@apache.org>.
On 05/12/2010 08:30 PM, Aaron Crow wrote:
> I may have a better idea of what caused the trouble. I way, WAY
> underestimated the number of nodes we collect over time. Right now we're at
> 1.9 million. This isn't a bug of our application; it's actually a feature
> (but perhaps an ill-conceived one).
>
> A most recent snapshot from a Zookeeper db is 227MB. If I scp it over to one
> of the other Zookeeper hosts, it takes about 4 seconds.
>

Nice. You probably hold the record for largest (znode count) production 
ZK repo. Largest I've heard of at least.

> Now, there are some things I can do to limit the number of nodes we collect.
> My question is, how deadly could this node size be for us? Patrick mentioned
> to me that he's run Zookeeper with this many nodes, but you need to be
> careful about tuning. We're currently running with the recommended JVM
> settings (see below). We're using different drives for the 2 different kinds
> of data dirs that Zookeeper needs. We may also have the option of running on
> a 64 bit OS with added RAM, if it's worth it. What about timeout settings?
> I'm copying in our current settings below, are those ok?
>

ALA you have enough memory/disk/IO you should be ok. Are you monitoring 
the operation latency on the servers? (via 4letter words, such as "stat"?)

You might increase the init/sync limits a bit to ensure that the 
followers have enough time to d/l the snapshot, deserialize it, and get 
setup with the leader (if this takes too long the quorum will fail and 
reelect a new leader, which might happen indefinitely).

> Or should we just figure out how to keep our node count much lower? And how
> low is "definitely pretty safe"?
>

There's really no "max" - it's just dependent on your resources. Memory 
in particular.

You should turn on incremental GC mode though (-XX:+CMSIncrementalMode), 
otw large GC pauses will wreck your latencies. Checkout this link 
(below), verbose gc is also useful to track down issues later (if 
something bad happens you can use it to rule out/in GC as an issue)

http://java.sun.com/docs/hotspot/gc5.0/gc_tuning_5.html#0.0.0.0.Incremental%20mode%7Coutline

Regards,

Patrick

>
> === some current settings ===
> -XX:ParallelGCThreads=8 -XX:+UseConcMarkSweepGC -Xms2560m -Xmx2560m
> tickTime=2000
> initLimit=10
> syncLimit=5
> ====================
>
>
> Many thanks in advance for any good advice.
> Aaron
>
>
> On Wed, Apr 28, 2010 at 10:47 PM, Patrick Hunt<ph...@apache.org>  wrote:
>
>> Hi Aaron, some questions/comments below:
>>
>>
>> On 04/28/2010 06:29 PM, Aaron Crow wrote:
>>
>>> We were running version 3.2.2 for about a month and it was working well
>>> for
>>> us. Then late this past Saturday night, our cluster went pathological. One
>>> of the 3 ZK servers spewed many WARNs (see below), and the other 2 servers
>>> were almost constantly pegging the CPU. All three servers are on separate
>>> machines. From what we could tell, the machines were fine... networking
>>> fine, disk fine, etc. The ZK clients were completely unable to complete
>>> their connections to ZK.
>>>
>>>
>> These machines are local (not wan) connected then? What OS and java version
>> are you using?
>>
>> Do you see any FATAL or ERROR level messages in the logs?
>>
>> It would help to look at your zk config files for these servers. Could you
>> provide (you might want to create a JIRA first, then just attach configs and
>> other details/collateral to that, easier than dealing with email)
>>
>> If you have logs for the time period and can share that would be most
>> useful. (again, gzip and attach to the jira)
>>
>>
>>   We tried all sorts of restarts, running zkCleanup, etc. We even completely
>>> shut down our clients... and the pathology continued. Our workaround was
>>> to
>>> do an urgent upgrade to version 3.3.0. The new ZK cluster with 3.3.0 has
>>> been running well for us... so far...
>>>
>>>
>> Off hand and with the data we have so far nothing sticks out that 3.3 would
>> have resolved (JIRA is conveniently down for the last hour or so so I can't
>> review right now). Although there were some changes to reduce memory
>> consumption (see below).
>>
>>
>>   I realize that, sadly, this message doesn't contain nearly enough details
>>> to
>>> trace exactly what happened. I guess I'm wondering if anyone has seen this
>>> general scenario, and/or knows how to prevent? Is there anything we might
>>> be
>>> doing client side to trigger this? Our application level request frequency
>>> is maybe a few requests to Zookeeper per second, times 5 clients
>>> applications. If we detect a SESSION EXPIRED, we do a simple "create new
>>> client and use that instead". And we were seeing this happen occasionally.
>>>
>>>
>> What are the client doing? Do you have a large number/size of znodes?
>>
>> Do you see any OutOfMemoryError in the logs?
>>
>> Could the ZK server java process be swapping? Are you monitoring GC,
>> perhaps large GC pauses are happening?
>>
>> I have a suspicion that one of a few things might be happening. I see the
>> following in your original email:
>>
>>
>>> :FollowerHandler@302] - Sending snapshot last zxid of peer is
>> 0xd0007d66d
>>> zxid of leader is 0xf00000000
>>> 2010-04-24 23:06:03,254 - ERROR [FollowerHandler-/10.0.10.116:34405
>>> :FollowerHandler@415] - Unexpected exception causing shutdown while sock
>>> still open
>>> java.net.SocketException: Broken pipe
>>>           at java.net.SocketOutputStream.socketWrite0(Native Method)
>>>           at
>>
>> ^^^^ this log section is saying that a ZK server (follower) connected to
>> the elected leader and started to download (leader is sending here) the
>> latest ZK snapshot (db) to the follower while it is recovering. However
>> before the follower fully downloads the snapshot the connection (pipe) is
>> broken and this operation fails. At this point the leader itself will
>> probably give up the lead, a new election will happen, and it might be the
>> case that this same problem occurs - the follower tries to download the
>> snapshot but it again fails (and this loops forever)
>>
>> This issue can happen if the servers are slowed, either by slow
>> interconnects (wan), network connectivity/perf problems, swapping of the
>> JVM, and most commonly GC in the VM.
>>
>> See this recent case:
>> http://www.mail-archive.com/zookeeper-dev@hadoop.apache.org/msg08083.html
>>
>> What is the size of your snapshot file? (you can see this in your zookeeper
>> datadirectory, unless you wiped this during your attempt to
>> cleanup/recover/upgrade?).
>>
>> Seeing your config file, and getting a sense of the snapshot file size will
>> give us more insight.
>>
>> Patrick
>>
>>
>>   Many, many thanks in advance for any insights or advice.
>>>
>>>
>>> Example log output from the ZK server that spewed many WARNs:
>>>
>>> ===
>>>
>>> 2010-04-24 22:31:24,551 - WARN
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@318] - Exception when
>>> following
>>> the leader
>>> java.net.SocketTimeoutException: Read timed out
>>> at java.net.SocketInputStream.socketRead0(Native Method)
>>> at java.net.SocketInputStream.read(SocketInputStream.java:129)
>>> at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>> at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
>>> at java.io.DataInputStream.readInt(DataInputStream.java:370)
>>> at org.apache.jute.BinaryInputArchive.readInt(BinaryInputArchive.java:63)
>>> at
>>>
>>> org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:66)
>>> at
>>> org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:108)
>>> at
>>> org.apache.zookeeper.server.quorum.Follower.readPacket(Follower.java:114)
>>> at
>>>
>>> org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:193)
>>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:525)
>>> 2010-04-24 22:31:24,552 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@436] - shutdown called
>>> java.lang.Exception: shutdown Follower
>>> at org.apache.zookeeper.server.quorum.Follower.shutdown(Follower.java:436)
>>> at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:529)
>>> 2010-04-24 22:31:24,552 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:QuorumPeer@514] - LOOKING
>>>
>>> ...
>>>
>>> 2010-04-24 22:31:27,257 - WARN
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>>> - Exception causing close of session 0x0 due to java.io.IOException:
>>> ZooKeeperServer not running
>>> 2010-04-24 22:31:27,257 - INFO
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>>> - closing session:0x0 NIOServerCnxn:
>>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>>> 127.0.0.1:44589]
>>> 2010-04-24 22:31:27,705 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:FastLeaderElection@706] -
>>> Notification:
>>> 3, 60129542145, 2, 1, LOOKING, LEADING, 3
>>> 2010-04-24 22:31:27,705 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:QuorumPeer@523] - FOLLOWING
>>> 2010-04-24 22:31:27,705 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:ZooKeeperServer@160] - Created server
>>> 2010-04-24 22:31:27,706 - INFO
>>>   [QuorumPeer:/0:0:0:0:0:0:0:0:2181:Follower@147] - Following sum08/
>>> 10.0.10.118:2888
>>> 2010-04-24 22:31:29,040 - WARN
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>>> - Exception causing close of session 0x0 due to java.io.IOException:
>>> ZooKeeperServer not running
>>> 2010-04-24 22:31:29,040 - INFO
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>>> - closing session:0x0 NIOServerCnxn:
>>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>>> 127.0.0.1:44591]
>>> 2010-04-24 22:31:31,069 - WARN
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>>> - Exception causing close of session 0x0 due to java.io.IOException:
>>> ZooKeeperServer not running
>>> 2010-04-24 22:31:31,069 - INFO
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@857]
>>> - closing session:0x0 NIOServerCnxn:
>>> java.nio.channels.SocketChannel[connected local=/127.0.0.1:2181remote=/
>>> 127.0.0.1:44592]
>>> 2010-04-24 22:31:32,589 - WARN
>>>   [NIOServerCxn.Factory:2181:NIOServerCnxn@518]
>>> - Exception causing close of session 0x0 due to java.io.IOException:
>>> ZooKeeperServer not running
>>>
>>> ===
>>>
>>> Example log output from a server that was pegging CPU:
>>>
>>> 010-04-24 23:05:27,911 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>> 2010-04-24 23:05:27,912 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>> 2010-04-24 23:05:27,910 - WARN  [Thread-15:QuorumCnxManager$SendWorker@539
>>> ]
>>> - Interrupted while waiting for message on queue
>>> java.lang.InterruptedException
>>>          at
>>>
>>> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.reportInterruptAfterWait(AbstractQueuedSynchronizer.java:1899)
>>>          at
>>>
>>> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:1934)
>>>          at
>>> java.util.concurrent.ArrayBlockingQueue.take(ArrayBlockingQueue.java:317)
>>>          at
>>>
>>> org.apache.zookeeper.server.quorum.QuorumCnxManager$SendWorker.run(QuorumCnxManager.java:533)
>>> 2010-04-24 23:05:27,912 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>> 2010-04-24 23:05:27,912 - WARN  [Thread-16:QuorumCnxManager$RecvWorker@623
>>> ]
>>> - Connection broken:
>>> java.nio.channels.AsynchronousCloseException
>>>          at
>>>
>>> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:185)
>>>          at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:263)
>>>          at
>>>
>>> org.apache.zookeeper.server.quorum.QuorumCnxManager$RecvWorker.run(QuorumCnxManager.java:594)
>>> 2010-04-24 23:05:27,912 - WARN  [Thread-15:QuorumCnxManager$SendWorker@554
>>> ]
>>> - Send worker leaving thread
>>>
>>> ...
>>>
>>> 010-04-24 23:05:51,952 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>> 2010-04-24 23:05:57,148 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>> 2010-04-24 23:06:03,243 - INFO  [FollowerHandler-/10.0.10.116:34405
>>> :FollowerHandler@227] - Follower sid: 1 : info :
>>> org.apache.zookeeper.server.quorum.QuorumPeer$QuorumServer@364641
>>> 2010-04-24 23:06:03,244 - WARN  [FollowerHandler-/10.0.10.116:34405
>>> :FollowerHandler@302] - Sending snapshot last zxid of peer is 0xd0007d66d
>>> zxid of leader is 0xf00000000
>>> 2010-04-24 23:06:03,254 - ERROR [FollowerHandler-/10.0.10.116:34405
>>> :FollowerHandler@415] - Unexpected exception causing shutdown while sock
>>> still open
>>> java.net.SocketException: Broken pipe
>>>          at java.net.SocketOutputStream.socketWrite0(Native Method)
>>>          at
>>> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
>>>          at java.net.SocketOutputStream.write(SocketOutputStream.java:136)
>>>          at
>>> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
>>>          at
>>> java.io.BufferedOutputStream.write(BufferedOutputStream.java:109)
>>>          at java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
>>>          at
>>> org.apache.jute.BinaryOutputArchive.writeLong(BinaryOutputArchive.java:59)
>>>          at
>>> org.apache.zookeeper.server.DataNode.serialize(DataNode.java:129)
>>>          at
>>>
>>> org.apache.jute.BinaryOutputArchive.writeRecord(BinaryOutputArchive.java:123)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:917)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serializeNode(DataTree.java:929)
>>>          at
>>> org.apache.zookeeper.server.DataTree.serialize(DataTree.java:979)
>>>          at
>>>
>>> org.apache.zookeeper.server.util.SerializeUtils.serializeSnapshot(SerializeUtils.java:104)
>>>          at
>>>
>>> org.apache.zookeeper.server.ZooKeeperServer.serializeSnapshot(ZooKeeperServer.java:272)
>>>          at
>>>
>>> org.apache.zookeeper.server.quorum.FollowerHandler.run(FollowerHandler.java:307)
>>> 2010-04-24 23:06:03,255 - WARN  [FollowerHandler-/10.0.10.116:34405
>>> :FollowerHandler@421] - ******* GOODBYE /10.0.10.116:34405 ********
>>> 2010-04-24 23:06:03,550 - INFO  [WorkerReceiver
>>> Thread:FastLeaderElection$Messenger$WorkerReceiver@254] - Sending new
>>> notification.
>>>
>>>
>