[ https://issues.apache.org/jira/browse/ZOOKEEPER-1549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13465725#comment-13465725 ]
Flavio Junqueira commented on ZOOKEEPER-1549: --------------------------------------------- I was thinking that it would be ok to remove that takeSnapshot() call from loadData(). It is incorrect as we have been discussing, and it doesn't seem to break anything, although a couple of tests failed when I gave it a try. I haven't looked the reasons for the tests to fail; I'll do it next. [~fanster.z] Which version of ZooKeeper are you using? The case I see in trunk looks like this: {noformat} case Leader.UPTODATE: if (!snapshotTaken) { // true for the pre v1.0 case zk.takeSnapshot(); self.setCurrentEpoch(newEpoch); } {noformat} > Data inconsistency when follower is receiving a DIFF with a dirty snapshot > -------------------------------------------------------------------------- > > Key: ZOOKEEPER-1549 > URL: https://issues.apache.org/jira/browse/ZOOKEEPER-1549 > Project: ZooKeeper > Issue Type: Bug > Components: quorum > Affects Versions: 3.4.3 > Reporter: Jacky007 > Priority: Critical > Attachments: case.patch > > > the trunc code (from ZOOKEEPER-1154?) cannot work correct if the snapshot is > not correct. > here is scenario(similar to 1154): > 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 2 > A,B restart, A is elected as the new leader, and A will load data and take a > clean snapshot(change 74 is in it), then send diff to B, but B died before > sync with A. A died later. > 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. > Problem: > The problem with the above sequence is that after truncate the log, A will > load the snapshot again which is not correct. > In 3.3 branch, FileTxnSnapLog.restore does not call listener(ZOOKEEPER-874), > the leader will send a snapshot to follower, it will not be a problem. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators For more information on JIRA, see: http://www.atlassian.com/software/jira