You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@zookeeper.apache.org by "Liu Shaohui (JIRA)" <ji...@apache.org> on 2015/01/04 11:13:34 UTC

[jira] [Created] (ZOOKEEPER-2101) Transaction larger than max buffer of jute makes zookeeper unavailable

Liu Shaohui created ZOOKEEPER-2101:
--------------------------------------

             Summary: Transaction larger than max buffer of jute makes zookeeper unavailable
                 Key: ZOOKEEPER-2101
                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2101
             Project: ZooKeeper
          Issue Type: Bug
          Components: jute
    Affects Versions: 3.4.4
            Reporter: Liu Shaohui


*Problem*
For multi operation, PrepRequestProcessor may produce a large transaction whose size may be larger than the max buffer size of jute. There is check of buffer size in readBuffer method  of BinaryInputArchive, but no check in writeBuffer method  of BinaryOutputArchive, which will cause that 

1, Leader can sync transaction to txn log and send the large transaction to the followers, but the followers failed to read the transaction and can't sync with leader.
{code}
2015-01-04,12:42:26,474 WARN org.apache.zookeeper.server.quorum.Learner: [myid:2] Exception when following the leader
java.io.IOException: Unreasonable length = 2054758
        at org.apache.jute.BinaryInputArchive.readBuffer(BinaryInputArchive.java:100)
        at org.apache.zookeeper.server.quorum.QuorumPacket.deserialize(QuorumPacket.java:85)
        at org.apache.jute.BinaryInputArchive.readRecord(BinaryInputArchive.java:108)
        at org.apache.zookeeper.server.quorum.Learner.readPacket(Learner.java:152)
        at org.apache.zookeeper.server.quorum.Follower.followLeader(Follower.java:85)
        at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:740)
2015-01-04,12:42:26,475 INFO org.apache.zookeeper.server.quorum.Learner: [myid:2] shutdown called
java.lang.Exception: shutdown Follower
        at org.apache.zookeeper.server.quorum.Follower.shutdown(Follower.java:166)
        at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:744)
{code}

2, The leader lose all followers, which trigger the leader election. The old leader will become leader again for it has up-to-date data.
{code}
2015-01-04,12:42:28,502 INFO org.apache.zookeeper.server.quorum.Leader: [myid:3] Shutting down
2015-01-04,12:42:28,502 INFO org.apache.zookeeper.server.quorum.Leader: [myid:3] Shutdown called
java.lang.Exception: shutdown Leader! reason: Only 1 followers, need 2
        at org.apache.zookeeper.server.quorum.Leader.shutdown(Leader.java:496)
        at org.apache.zookeeper.server.quorum.Leader.lead(Leader.java:471)
        at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:753)
{code}
3, The leader can not load the transaction from the txn log for the length of data is larger than the max buffer of jute.
{code}

2015-01-04,12:42:31,282 ERROR org.apache.zookeeper.server.quorum.QuorumPeer: [myid:3] Unable to load database on disk
java.io.IOException: Unreasonable length = 2054758
        at org.apache.jute.BinaryInputArchive.readBuffer(BinaryInputArchive.java:100)
        at org.apache.zookeeper.server.persistence.Util.readTxnBytes(Util.java:233)
        at org.apache.zookeeper.server.persistence.FileTxnLog$FileTxnIterator.next(FileTxnLog.java:602)
        at org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:157)
        at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:223)
        at org.apache.zookeeper.server.quorum.QuorumPeer.loadDataBase(QuorumPeer.java:417)
        at org.apache.zookeeper.server.quorum.QuorumPeer.getLastLoggedZxid(QuorumPeer.java:546)
        at org.apache.zookeeper.server.quorum.FastLeaderElection.getInitLastLoggedZxid(FastLeaderElection.java:690)
        at org.apache.zookeeper.server.quorum.FastLeaderElection.lookForLeader(FastLeaderElection.java:737)
        at org.apache.zookeeper.server.quorum.QuorumPeer.run(QuorumPeer.java:716)
{code}

The zookeeper service will be unavailable until we enlarge the jute.maxbuffer and restart zookeeper hbase cluster.

*Solution*
Add buffer size check in BinaryOutputArchive to avoid large transaction be written to log and sent to followers.

But I am not sure if there are side-effects of throwing an IOException in BinaryOutputArchive  and RequestProcessors




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)