[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-2845?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16360557#comment-16360557
 ] 

ASF GitHub Bot commented on ZOOKEEPER-2845:
-------------------------------------------

Github user mfenes commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/453#discussion_r167513290
  
    --- Diff: 
src/java/main/org/apache/zookeeper/server/quorum/LearnerHandler.java ---
    @@ -758,6 +760,11 @@ public boolean syncFollower(long peerLastZxid, 
ZKDatabase db, Leader leader) {
                     currentZxid = maxCommittedLog;
                     needOpPacket = false;
                     needSnap = false;
    +            } else if (peerLastEpoch != lastProcessedEpoch && 
!db.isInCommittedLog(peerLastZxid)) {
    +                //Be sure we do a snap, because if the epochs are not the 
same we don't know what
    +                // could have happened in between and it may take a TRUNC 
+ UPDATES to get them in SYNC
    +                LOG.debug("Will send SNAP to peer sid: {} epochs are too 
our of sync local 0x{} remote 0x{}",
    --- End diff --
    
    I think there is a typo here: "our of sync"


> 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)

Reply via email to