Uploaded image for project: 'Aurora'
  1. Aurora
  2. AURORA-1061

Scheduler cluster deadlock during leader election if zookeeper connectivity is interrupted

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.16.0
    • Reliability, Scheduler
    • None

    Description

      Observed in production: the scheduler cluster deadlocked with all nodes waiting for a leader.

      This manifested as 6 singleton_candidate_ nodes in ZooKeeper. Since there were 5 schedulers one was double-counted. Inspecting the contents, 2 nodes contained the same IP address.

      Restarting the scheduler with duplicated znodes allowed a new leader to take over, mitigating the issue.

      This is a bug in the leader election library. The scheduler created a new ephemeral znode after a connection loss with the zk ensemble without realizing that its session hadn't expired and thus its previous znode was still valid. What's worse, that previous znode was the winner of the election, making other schedulers sit idle as the leader waited for itself.

      Logs from the stuck leader confirm: it attempted to do a create, got a ConnectionLoss, then reconnected with the same session id (0xf4af4a49d4ef61c), created a new node (singleton_candidate_0000003471). Meanwhile the previous created ephemeral znode (singleton_candidate_0000003470) persisted as the session was kept alive.

      I0120 23:03:30.206 THREAD133 org.apache.zookeeper.ClientCnxn$SendThread.run: Client session timed out, have not heard from server in 6670ms for sessionid 0xf4af4a49d4ef61c, closing socket connection and attempting reconnect
      W0120 23:03:30.315 THREAD1 com.twitter.common.zookeeper.Group$2.get: Temporary error trying to join group at path: /twitter/service/meso
      s/prod/scheduler
      org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /twitter/service/mesos/prod/scheduler/singleton_candidate_
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:90)
              at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
              at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:643)
              at com.twitter.common.zookeeper.Group$ActiveMembership.join(Group.java:424)
              at com.twitter.common.zookeeper.Group$2.get(Group.java:273)
              at com.twitter.common.zookeeper.Group$2.get(Group.java:270)
              at com.twitter.common.util.BackoffHelper.doUntilResult(BackoffHelper.java:127)
              at com.twitter.common.zookeeper.Group.join(Group.java:270)
              at com.twitter.common.zookeeper.CandidateImpl.offerLeadership(CandidateImpl.java:128)
              at com.twitter.common.zookeeper.SingletonService.lead(SingletonService.java:165)
              at org.apache.aurora.scheduler.app.SchedulerMain.run(SchedulerMain.java:216)
              at com.twitter.common.application.AppLauncher.run(AppLauncher.java:102)
              at com.twitter.common.application.AppLauncher.launch(AppLauncher.java:181)
              at com.twitter.common.application.AppLauncher.launch(AppLauncher.java:142)
              at org.apache.aurora.scheduler.app.SchedulerMain.main(SchedulerMain.java:232)
      
      I0120 23:03:30.366 THREAD133 org.apache.zookeeper.ClientCnxn$SendThread.startConnect: Opening socket connection to server sdzookeeper.local.twitter.com/10.54.7.132:2181
      I0120 23:03:30.366 THREAD133 org.apache.zookeeper.ClientCnxn$SendThread.primeConnection: Socket connection established to sdzookeeper.local.twitter.com/10.54.7.132:2181, initiating session
      I0120 23:03:30.368 THREAD133 org.apache.zookeeper.ClientCnxn$SendThread.readConnectResult: Session establishment complete on server sdzookeeper.local.twitter.com/10.54.7.132:2181, sessionid = 0xf4af4a49d4ef61c, negotiated timeout = 10000
      I0120 23:03:31.683 THREAD1 com.twitter.common.zookeeper.Group$ActiveMembership.join: Set group member ID to singleton_candidate_0000003471
      I0120 23:07:22.548 THREAD134 com.twitter.common.zookeeper.CandidateImpl$4.onGroupChange: Candidate /twitter/service/mesos/prod/scheduler/singleton_candidate_0000003471 waiting for the next leader election, current voting: [singleton_candidate_0000003470, singleton_candidate_0000003471, singleton_candidate_0000003476, singleton_candidate_0000003474, singleton_candidate_0000003472, singleton_candidate_0000003473]
      

      Attachments

        Issue Links

          Activity

            People

              jsirois John Sirois
              kevints Kevin Sweeney
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: