Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
3.6.3
-
None
-
None
-
None
Description
Here is the bug triggering process:
- A cluster with three nodes: zk1, zk2 and zk3. zk3 is the leader.
- client create a znode "/bug" with value "bad"
- client update znode "/bug" to value "good"
- zk1 crashes before receiving proposal for leader for the request in step 3.
- "/bug" is modified to "good"
- zk1 was restarted
- another client connects to zk1, reads "/bug" and gets "bad"
- zk1 finish synchronization with current leader, and then modify "/bug" to "good".
The problem is that zk1 should be accessed by a client when it finish synchronization with current leader in case of a client reads bad data.
****************************************************************************************
The actual testing scenario is as following:
I have a cluster of 5 nodes: C1ZK1(172.30.0.2), C1ZK2(172.30.0.3), C1ZK3(172.30.0.4), C1ZK4(172.30.0.5), C1ZK5(172.30.0.6)
- 2022-03-24 22:51:40,246 [Client1] - INFO - build connection with zookeeper (client1 actuallly builds connection with C1ZK1)
- 2022-03-24 22:51:40,479 crash C1ZK4 before creating file "/home/zkuser/evaluation/zk-3.6.3/zkData/version-2/log.100000001" (I think this crash does not matter):
java.io.FileOutputStream.<init>(FileOutputStream.java:213), java.io.FileOutputStream.<init>(FileOutputStream.java:162), org.apache.zookeeper.server.persistence.FileTxnLog.append(FileTxnLog.java:287), org.apache.zookeeper.server.persistence.FileTxnSnapLog.append(FileTxnSnapLog.java:582), org.apache.zookeeper.server.ZKDatabase.append(ZKDatabase.java:641), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:181), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:0
- 2022-03-24 22:51:40,761 [Client1] - INFO - created znode "/bug" "hello"
- 2022-03-24 22:51:40,869 [Client1] - INFO - set znode "/bug" "nice"
- 2022-03-24 22:51:40,915 [Client1] - INFO - read znode "/bug" is "nice"
- 2022-03-24 22:51:40,996 [Client1] - INFO - deleted znode "/bug"
- Client1 requests to create ephemeral znode "/eph"
- 2022-03-24 22:51:45,033 crash C1ZK1 before:
org.apache.zookeeper.server.quorum.QuorumPacket.serialize(QuorumPacket.java:68), org.apache.jute.BinaryOutputArchive.writeRecord(BinaryOutputArchive.java:126), org.apache.zookeeper.server.quorum.Learner.writePacketNow(Learner.java:194), org.apache.zookeeper.server.quorum.Learner.writePacket(Learner.java:186), org.apache.zookeeper.server.quorum.SendAckRequestProcessor.processRequest(SendAckRequestProcessor.java:46), org.apache.zookeeper.server.SyncRequestProcessor.flush(SyncRequestProcessor.java:246), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:169), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:0)
- 2022-03-24 22:51:49,451 restart C1ZK1 before C1ZK2 write to file "/home/zkuser/evaluation/zk-3.6.3/zkData/version-2/log.100000001":
java.io.FileOutputStream.writeBytes(FileOutputStream.java), java.io.FileOutputStream.write(FileOutputStream.java:326), java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82), java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140), org.apache.zookeeper.server.persistence.FileTxnLog.append(FileTxnLog.java:293), org.apache.zookeeper.server.persistence.FileTxnSnapLog.append(FileTxnSnapLog.java:582), org.apache.zookeeper.server.ZKDatabase.append(ZKDatabase.java:641), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:181), org.apache.zookeeper.server.SyncRequestProcessor.run(SyncRequestProcessor.java:0)
- 2022-03-24 22:51:56,744 [Client2] - INFO - build connection with zookeeper
- 2022-03-24 22:51:56,876 [Client2] - INFO - cannot read ephemeral znode "/eph", got "KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /eph"
- When we connect with every alive node in the cluster and read data respectively, we get
2022-03-24 22:52:14,663 [ZKChecker] - INFO - server C1ZK3:11181 and server C1ZK1:11181 have different number of znodes:[/zookeeper/quota, /zookeeper] and [/zookeeper/quota, /eph, /zookeeper]
- Then I killed all the nodes in the cluster
The file "log.100000001" in C1ZK1 is :
ZooKeeper Transactional Log File with dbid 0 txnlog format version 2 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x0 zxid 0x100000001 createSession 15000 2,1371985504 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x1 zxid 0x100000002 create '/bug,#68656c6c6f,v{s{31,s{'world,'anyone}}},F,1 2,6620487461 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x2 zxid 0x100000003 setData '/bug,#6e696365,1 2,5588659454 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x4 zxid 0x100000004 delete '/bug 2,2910118891 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x5 zxid 0x100000005 create '/eph,#657068656d,v{s{31,s{'world,'anyone}}},T,2 2,7205004542 EOF reached after 5 txns.
And txn logs in other nodes are:
ZooKeeper Transactional Log File with dbid 0 txnlog format version 2 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x0 zxid 0x100000001 createSession 15000 2,1371985504 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x1 zxid 0x100000002 create '/bug,#68656c6c6f,v{s{31,s{'world,'anyone}}},F,1 2,6620487461 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x2 zxid 0x100000003 setData '/bug,#6e696365,1 2,5588659454 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x4 zxid 0x100000004 delete '/bug 2,2910118891 22-3-24 22:51:40 session 0x3035dd9dccf0000 cxid 0x5 zxid 0x100000005 create '/eph,#657068656d,v{s{31,s{'world,'anyone}}},T,2 2,7205004542 22-3-24 22:51:57 session 0x5035dda2c040000 cxid 0x0 zxid 0x200000001 createSession 15000 2,7205004542 22-3-24 22:51:57 session 0x5035dda2c040000 cxid 0x1 zxid 0x200000002 closeSession v{} 2,7205004542 EOF reached after 7 txns.