[
https://issues.apache.org/jira/browse/ZOOKEEPER-1549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13566235#comment-13566235
]
Thawan Kooburat commented on ZOOKEEPER-1549:
--------------------------------------------
High-level Description.
Leader:
1. The leader loads the db from snapshot only. If the quorum hasn't formed, it
will synchronize with the learner using proposals read directly from txnlog.
2. When the quorum is formed (received sufficient NEWLEADER_ACK). The leader
reply its txns and add them to committedLog. Any learner joining after this
point will only use committedLog for synchronization.
Learner:
1. Similar to the leader, the learner loads the db from snapshot only. Since it
send lastLoggedZxid to the leader as part of FOLLOWER/OBSERVERINFO it should
only get txn that it hasn't seen if it get a DIFF. Otherwise, it will get a
SNAP + DIFF
2. During synchronization (after received DIFF or SNAP packet), the learner
will get a stream of txns in form of prosposal + commit. The learner logs each
txn to txnlog only when it received a corresponding commit. This is because
the learner may receive outstanding proposals before getting UPTODATE if it
joins the quorum after it is formed.
3. When the learner received NEWLEADER, it flushes the txnlog to disk before
sending NEWLEADER_ACK back to the leader.
4. When the learner received UPTODATE, it flushed the txnlog again (in case it
see any more committed txn). Then, it replay txnlog that it originally seen
before joining the quorum and apply the committed txns that its received from
the leader (and currently in memory) to the db. It then sends UPTODATE_ACK back
to the leader but the leader is going to ignore this ACK since it is part of
the old protocol.
5. After sending UPTODATE_ACK, the learner starts up the server. For the
follower, it will have to process any outstanding proposal that it received
during synchronization phase, so that the leader will get ACK from the follower
(and the follower don't confuse if it see corresponding commit message from the
leader when it start processing request.
Db initialization:
1. When loading the snapshot, it also capture the snapshot zxid into a separate
variable. This is later used to during txn replay. I didn't want to overload
lastProcessedZxid since it may have other implication.
2. Last logged zxid is obtained in a way to guarantee that we don't run into
ZOOKEEPER-596.
3. I saw the logic that kill dead sessions on db loading. I am not sure why we
have this logic and don't know its implication toward data inconsistency.
4. No snapshot is taken (unless the learner get a snapshot) when the new quorum
form. I think we might want to add some logic to trigger snapshot in
background soon after the quorum is formed.
Side Note:
- One of the major complexity in the existing code on the learner side is due
to the fact that the learner may see outstanding proposal before it received
UPTODATE. I want to cover this case so that we can do rolling upgrade when
pushing this patch/release. The system should behave correctly if rolling
upgrade is done by upgrading all the followers first. Zab1_0Test is modified to
test this case.
- This patch also fixed ZOOKEEPER-1551
- The actual change on our internal branch is about half the size of the
submitted patch. Another half is a result of our refactoring work on learner
synchronization logic as part of ZOOKEEPER-1413. Since we currently use this in
our production and it has a better unit testing facility, so I use this
synchronization logic instead of the current one in trunk.
- Is there away to trigger a unit test run and submit a review request against
3.4 branch?
> 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
> Assignee: Thawan Kooburat
> Priority: Blocker
> Fix For: 3.5.0, 3.4.6
>
> Attachments: case.patch, ZOOKEEPER-1549-3.4.patch,
> ZOOKEEPER-1549-learner.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