[ https://issues.apache.org/jira/browse/ZOOKEEPER-4503?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
May updated ZOOKEEPER-4503: --------------------------- 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): {code:java} 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{code} # 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" was: 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 connects 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): {code:java} 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{code} # 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" > A restarted node can be accessed before it finishing synchronization with > leader > -------------------------------------------------------------------------------- > > Key: ZOOKEEPER-4503 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4503 > Project: ZooKeeper > Issue Type: Bug > Affects Versions: 3.6.3 > Reporter: May > Priority: Major > > 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): > {code:java} > 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{code} > # 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" -- This message was sent by Atlassian Jira (v8.20.1#820001)