[ https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16367561#comment-16367561 ]
ASF GitHub Bot commented on ZOOKEEPER-2845: ------------------------------------------- Github user revans2 commented on a diff in the pull request: https://github.com/apache/zookeeper/pull/453#discussion_r168807914 --- Diff: src/java/test/org/apache/zookeeper/server/quorum/QuorumPeerMainTest.java --- @@ -888,4 +888,127 @@ public void testWithOnlyMinSessionTimeout() throws Exception { maxSessionTimeOut, quorumPeer.getMaxSessionTimeout()); } + @Test + public void testTxnAheadSnapInRetainDB() throws Exception { + // 1. start up server and wait for leader election to finish + ClientBase.setupTestEnv(); + final int SERVER_COUNT = 3; + final int clientPorts[] = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + sb.append("server." + i + "=127.0.0.1:" + PortAssignment.unique() + ":" + PortAssignment.unique() + ";" + clientPorts[i] + "\n"); + } + String quorumCfgSection = sb.toString(); + + MainThread mt[] = new MainThread[SERVER_COUNT]; + ZooKeeper zk[] = new ZooKeeper[SERVER_COUNT]; + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], quorumCfgSection); + mt[i].start(); + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + + waitForAll(zk, States.CONNECTED); + + // we need to shutdown and start back up to make sure that the create session isn't the first transaction since + // that is rather innocuous. + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].shutdown(); + } + + waitForAll(zk, States.CONNECTING); + + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i].start(); + // Recreate a client session since the previous session was not persisted. + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + } + + waitForAll(zk, States.CONNECTED); + + // 2. kill all followers + int leader = -1; + Map<Long, Proposal> outstanding = null; + for (int i = 0; i < SERVER_COUNT; i++) { + if (mt[i].main.quorumPeer.leader != null) { + leader = i; + outstanding = mt[leader].main.quorumPeer.leader.outstandingProposals; + // increase the tick time to delay the leader going to looking + mt[leader].main.quorumPeer.tickTime = 10000; + } + } + LOG.warn("LEADER {}", leader); + + for (int i = 0; i < SERVER_COUNT; i++) { + if (i != leader) { + mt[i].shutdown(); + } + } + + // 3. start up the followers to form a new quorum + for (int i = 0; i < SERVER_COUNT; i++) { + if (i != leader) { + mt[i].start(); + } + } + + // 4. wait one of the follower to be the leader + for (int i = 0; i < SERVER_COUNT; i++) { + if (i != leader) { + // Recreate a client session since the previous session was not persisted. + zk[i] = new ZooKeeper("127.0.0.1:" + clientPorts[i], ClientBase.CONNECTION_TIMEOUT, this); + waitForOne(zk[i], States.CONNECTED); + } + } + + // 5. send a create request to leader and make sure it's synced to disk, + // which means it acked from itself + try { + zk[leader].create("/zk" + leader, "zk".getBytes(), Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + Assert.fail("create /zk" + leader + " should have failed"); + } catch (KeeperException e) { + } + + // just make sure that we actually did get it in process at the + // leader + Assert.assertTrue(outstanding.size() == 1); + Proposal p = (Proposal) outstanding.values().iterator().next(); --- End diff -- removed the cast > Data inconsistency issue due to retain database in leader election > ------------------------------------------------------------------ > > Key: ZOOKEEPER-2845 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2845 > Project: ZooKeeper > Issue Type: Bug > Components: quorum > Affects Versions: 3.4.10, 3.5.3, 3.6.0 > Reporter: Fangmin Lv > Assignee: Robert Joseph Evans > Priority: Critical > > In ZOOKEEPER-2678, the ZKDatabase is retained to reduce the unavailable time > during leader election. In ZooKeeper ensemble, it's possible that the > snapshot is ahead of txn file (due to slow disk on the server, etc), or the > txn file is ahead of snapshot due to no commit message being received yet. > If snapshot is ahead of txn file, since the SyncRequestProcessor queue will > be drained during shutdown, the snapshot and txn file will keep consistent > before leader election happening, so this is not an issue. > But if txn is ahead of snapshot, it's possible that the ensemble will have > data inconsistent issue, here is the simplified scenario to show the issue: > Let's say we have a 3 servers in the ensemble, server A and B are followers, > and C is leader, and all the snapshot and txn are up to T0: > 1. A new request reached to leader C to create Node N, and it's converted to > txn T1 > 2. Txn T1 was synced to disk in C, but just before the proposal reaching out > to the followers, A and B restarted, so the T1 didn't exist in A and B > 3. A and B formed a new quorum after restart, let's say B is the leader > 4. C changed to looking state due to no enough followers, it will sync with > leader B with last Zxid T0, which will have an empty diff sync > 5. Before C take snapshot it restarted, it replayed the txns on disk which > includes T1, now it will have Node N, but A and B doesn't have it. > Also I included the a test case to reproduce this issue consistently. > We have a totally different RetainDB version which will avoid this issue by > doing consensus between snapshot and txn files before leader election, will > submit for review. -- This message was sent by Atlassian JIRA (v7.6.3#76005)