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

Vishal Kathuria commented on ZOOKEEPER-1154:
--------------------------------------------

I connected the debugger to B and C and paused all threads to make sure B and C 
don't get a chance to log the request before they crash. I waited a few secs to 
give A enough time to log the request though.

> Data inconsistency when the node(s) with the highest zxid is not present at 
> the time of leader election
> -------------------------------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-1154
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1154
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: quorum
>    Affects Versions: 3.3.3
>            Reporter: Vishal Kathuria
>            Priority: Critical
>             Fix For: 3.4.0
>
>   Original Estimate: 504h
>  Remaining Estimate: 504h
>
> If a participant with the highest zxid (lets call it A) isn't present during 
> leader election, a participant with a lower zxid (say B) might be chosen as a 
> leader. When A comes up, it will replay the log with that higher zxid. The 
> change that was in that higher zxid will only be visible to the clients 
> connecting to the participant A, but not to other participants.
> I was able to reproduce this problem by
> 1. connect debugger to B and C and suspend them, so they don't write anything
> 2. Issue an update to the leader A.
> 3. After a few seconds, crash all servers (A,B,C)
> 4. Start B and C, let the leader election take place
> 5. Start A.
> 6. You will find that the update done in step 2 is visible on A but not on 
> B,C, hence the inconsistency.
> Below is a more detailed analysis of what is happening in the code.
> Initial Condition
> 1.    Lets say there are three nodes in the ensemble A,B,C with A being the 
> leader
> 2.    The current epoch is 7. 
> 3.    For simplicity of the example, lets say zxid is a two digit number, 
> with epoch being the first digit.
> 4.    The zxid is 73
> 5.    All the nodes have seen the change 73 and have persistently logged it.
> Step 1
> Request with zxid 74 is issued. The leader A writes it to the log but there 
> is a crash of the entire ensemble and B,C never write the change 74 to their 
> log.
> Step 3
> B,C restart, A is still down
> B,C form the quorum
> B is the new leader. Lets say  B minCommitLog is 71 and maxCommitLog is 73
> epoch is now 8, zxid is 80
> Request with zxid 81 is successful. On B, minCommitLog is now 71, 
> maxCommitLog is 81
> Step 4
> A starts up. It applies the change in request with zxid 74 to its in-memory 
> data tree
> A contacts B to registerAsFollower and provides 74 as its ZxId
> Since 71<=74<=81, B decides to send A the diff. B will send to A the proposal 
> 81.
> Problem:
> The problem with the above sequence is that A's data tree has the update from 
> request 74, which is not correct. Before getting the proposals 81, A should 
> have received a trunc to 73. I don't see that in the code. If the 
> maxCommitLog on B hadn't bumped to 81 but had stayed at 73, that case seems 
> to be fine.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to