Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
None
-
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
- depends upon
-
AURORA-1669 Kill twitter/commons ZK libs when Curator replacements are vetted
- Resolved
-
AURORA-1468 Replace org.apache.aurora.commons.zookeeper with a Apache Curator
- Resolved
- is depended upon by
-
AURORA-1661 Scheduler leader failed to re-announce itself after ZK name was changed
- Resolved