You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@aurora.apache.org by "Kevin Sweeney (JIRA)" <ji...@apache.org> on 2015/01/26 21:34:34 UTC

[jira] [Created] (AURORA-1061) Scheduler cluster deadlock during leader election if zookeeper connectivity is interrupted

Kevin Sweeney created AURORA-1061:
-------------------------------------

             Summary: Scheduler cluster deadlock during leader election if zookeeper connectivity is interrupted
                 Key: AURORA-1061
                 URL: https://issues.apache.org/jira/browse/AURORA-1061
             Project: Aurora
          Issue Type: Bug
          Components: Reliability, Scheduler
            Reporter: Kevin Sweeney


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.

{noformat}
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]
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)