[
https://issues.apache.org/jira/browse/ZOOKEEPER-4712?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andor Molnar resolved ZOOKEEPER-4712.
-------------------------------------
Fix Version/s: 3.10.0
3.9.3
Resolution: Fixed
Issue resolved by pull request 2154
[https://github.com/apache/zookeeper/pull/2154]
> Follower.shutdown() and Observer.shutdown() do not correctly shutdown the
> syncProcessor, which may lead to data inconsistency
> -----------------------------------------------------------------------------------------------------------------------------
>
> Key: ZOOKEEPER-4712
> URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4712
> Project: ZooKeeper
> Issue Type: Bug
> Components: quorum, server
> Affects Versions: 3.5.10, 3.6.3, 3.7.0, 3.8.0, 3.7.1, 3.6.4, 3.9.0, 3.8.1,
> 3.9.1
> Reporter: Sirius
> Priority: Critical
> Labels: pull-request-available
> Fix For: 3.10.0, 3.9.3
>
> Time Spent: 2h 50m
> Remaining Estimate: 0h
>
> Follower.shutdown() and Observer.shutdown() do not correctly shutdown the
> syncProcessor. It may lead to potential data inconsistency (see {*}Potential
> Risk{*}).
>
> A follower / observer will invoke syncProcessor.shutdown() in
> LearnerZooKeeperServer.shutdown() / ObserverZooKeeperServer.shutdown(),
> respectively.
> However, after the
> [FIX|https://github.com/apache/zookeeper/commit/efbd660e1c4b90a8f538f2cccb5dcb7094cf9a22]
> of ZOOKEEPER-3642, Follower.shutdown() / Observer.shutdown() will not invoke
> LearnerZooKeeperServer.shutdown() / ObserverZooKeeperServer.shutdown()
> anymore.
>
> h2. Method Invocation Path
> h5. Version 3.8.1 / 3.8.0 / 3.7.1 / 3.7.0 / 3.6.4 / 3.6.3 / 3.5.10 ...
> * *(Buggy)* Observer.shutdown() -> Learner.shutdown() ->
> ZooKeeperServer.shutdown(boolean)
> * *(Buggy)* Follower.shutdown() -> Learner.shutdown() ->
> ZooKeeperServer.shutdown(boolean)
> * (For comparison) Leader.shutdown(String) -> LeaderZooKeeper.shutdown() ->
> ZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown(boolean)
>
> h5. For comparison, in version 3.4.X,
> * Observer.shutdown() -> Learner.shutdown() ->
> {*}ObserverZooKeeperServer.shutdown() -{*}> ZooKeeperServer.shutdown() ->
> ZooKeeperServer.shutdown(boolean)
> * Follower.shutdown() -> Learner.shutdown() ->
> {*}FollowerZooKeeperServer.shutdown() -{*}> ZooKeeperServer.shutdown() ->
> ZooKeeperServer.shutdown(boolean)
>
> h5. Or, in version 3.6.0,
> * Observer.shutdown() -> Learner.shutdown() ->
> {*}LearnerZooKeeperServer.shutdown() -{*}> ZooKeeperServer.shutdown() ->
> ZooKeeperServer.shutdown(boolean)
> * Follower.shutdown() -> Learner.shutdown() ->
> {*}Learner{*}{*}ZooKeeperServer.shutdown() -{*}> ZooKeeperServer.shutdown()
> -> ZooKeeperServer.shutdown(boolean)
>
> h2. Code Details
> Take version 3.8.0 as an example.
> In Follower.shutdown() :
> {code:java}
> public void shutdown() {
> LOG.info("shutdown Follower");
> + // invoke Learner.shutdown()
> super.shutdown();
> } {code}
>
> In Learner.java:
> {code:java}
> public void shutdown() {
> ...
> // shutdown previous zookeeper
> if (zk != null) {
> // If we haven't finished SNAP sync, force fully shutdown
> // to avoid potential inconsistency
> + // This will invoke ZooKeeperServer.shutdown(boolean),
> + // which will not shutdown syncProcessor
> + // Before the fix of ZOOLEEPER-3642,
> + // FollowerZooKeeperServer.shutdown() will be invoked here
> zk.shutdown(self.getSyncMode().equals(QuorumPeer.SyncMode.SNAP));
> }
> } {code}
>
> In ZooKeeperServer.java:
> {code:java}
> public synchronized void shutdown(boolean fullyShutDown) {
> ...
> if (firstProcessor != null) {
> + // For a follower, this will not shutdown its syncProcessor.
> firstProcessor.shutdown();
> }
> ...
> } {code}
>
> In expectation, Follower.shutdown() should invoke
> LearnerZooKeeperServer.shutdown() to shutdown the syncProcessor:
> {code:java}
> public synchronized void shutdown() {
> ...
> try {
> + // shutdown the syncProcessor here
> if (syncProcessor != null) {
> syncProcessor.shutdown();
> }
> } ...
> } {code}
> Observer.shutdown() has the similar problem.
>
> h2. Potential Risk
> When Follower.shutdown() is called, the follower's QuorumPeer thread may
> update the lastProcessedZxid for the election and recovery phase before its
> syncThread drains the pending requests and flushes them to disk.
> In consequence, this lastProcessedZxid is not the latest zxid in its log,
> leading to log inconsistency after the SYNC phase. (Similar to the symptoms
> of ZOOKEEPER-2845.)
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)