Uploaded image for project: 'ZooKeeper'
  1. ZooKeeper
  2. ZOOKEEPER-4712

Follower.shutdown() and Observer.shutdown() do not correctly shutdown the syncProcessor, which may lead to data inconsistency

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 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
    • 3.10.0, 3.9.3
    • quorum, server

    Description

      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 of ZOOKEEPER-3642, Follower.shutdown() / Observer.shutdown() will not invoke LearnerZooKeeperServer.shutdown() / ObserverZooKeeperServer.shutdown() anymore.

       

      Method Invocation Path

      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)

       

      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)

       

      Or, in version 3.6.0,
      • Observer.shutdown() -> Learner.shutdown() -> LearnerZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown(boolean)
      • Follower.shutdown() -> Learner.shutdown() -> LearnerZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown() -> ZooKeeperServer.shutdown(boolean)

       

      Code Details

      Take version 3.8.0 as an example.

      In Follower.shutdown() :

          public void shutdown() {
              LOG.info("shutdown Follower");
      +       // invoke Learner.shutdown()
              super.shutdown();   
          } 

       

      In Learner.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));          }
          } 

       

      In ZooKeeperServer.java:

          public synchronized void shutdown(boolean fullyShutDown) {
              ...
              if (firstProcessor != null) {
      +           // For a follower, this will not shutdown its syncProcessor.
                  firstProcessor.shutdown(); 
              }
              ...
          } 

       

      In expectation, Follower.shutdown() should invoke LearnerZooKeeperServer.shutdown() to shutdown the syncProcessor:

          public synchronized void shutdown() {
              ...
              try {
      +           // shutdown the syncProcessor here
                  if (syncProcessor != null) {
                      syncProcessor.shutdown();     
                  }
              } ...
          } 

      Observer.shutdown() has the similar problem.

       

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

       

      Attachments

        Issue Links

          Activity

            People

              jonmv Jon Marius Venstad
              ouyang Sirius
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 4h 10m
                  4h 10m