mesos / chronos

Fault tolerant job scheduler for Mesos which handles dependencies and ISO8601 based schedules
http://mesos.github.io/chronos/
Apache License 2.0
4.39k stars 528 forks source link

If adding a vertex that already exists after leader re-election, Chronos eats all resource offers #290

Closed FxChiP closed 9 years ago

FxChiP commented 9 years ago

I ran into an issue recently that seems to involve Chronos losing and subsequently regaining connection to multiple ZooKeepers. After having lost and reconnected, Chronos loads all of the tasks currently written to ZooKeeper (or Mesos?). In this case, it purged and then filtered a lot of old tasks like so (task name edited):

[2014-10-28 01:54:40,744] INFO Purging entry 'T_ct:1414354200000:0:<task>' via: org.apache.mesos.state.ZooKeeperState (com.airbnb.scheduler.state.MesosStatePersistenceStore:167)
[2014-10-28 01:54:40,747] INFO Filtering out old task 'ct:1414354200000:0:<task>' overdue by '-107078074' ms and removing from store. (com.airbnb.scheduler.jobs.TaskUtils$:103)

Next, it loads the jobs it currently has on record:

[2014-10-28 01:54:40,747] INFO Loading jobs (com.airbnb.scheduler.jobs.JobScheduler:679)
[2014-10-28 01:54:40,751] INFO Registering jobs:3 (com.airbnb.scheduler.jobs.JobUtils$:74)

Finally, it adds vertexes to a DAG (I'm guessing for the dependency graph?) (note: job name edited):

[2014-10-28 01:54:40,752] WARN Adding vertex:<job> (com.airbnb.scheduler.graph.JobGraph:65)
[2014-10-28 01:54:40,753] ERROR Listener (com.airbnb.scheduler.jobs.JobScheduler$$anon$2@397314a4) threw an exception (org.apache.curator.framework.listen.ListenerContainer:96)
java.lang.IllegalArgumentException: requirement failed: Vertex already exists in graph <job>
        at scala.Predef$.require(Predef.scala:233)
        at com.airbnb.scheduler.graph.JobGraph.addVertex(JobGraph.scala:66)
        at com.airbnb.scheduler.jobs.JobScheduler$$anonfun$registerJob$3.apply(JobScheduler.scala:178)
        at com.airbnb.scheduler.jobs.JobScheduler$$anonfun$registerJob$3.apply(JobScheduler.scala:177)
        at scala.collection.immutable.List.foreach(List.scala:318)
        at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:32)
        at scala.collection.mutable.ListBuffer.foreach(ListBuffer.scala:45)
        at com.airbnb.scheduler.jobs.JobScheduler.registerJob(JobScheduler.scala:176)
        at com.airbnb.scheduler.jobs.JobUtils$.loadJobs(JobUtils.scala:75)
        at com.airbnb.scheduler.jobs.JobScheduler.onElected(JobScheduler.scala:680)
        at com.airbnb.scheduler.jobs.JobScheduler$$anon$2.isLeader(JobScheduler.scala:645)
        at org.apache.curator.framework.recipes.leader.LeaderLatch$9.apply(LeaderLatch.java:644)
        at org.apache.curator.framework.recipes.leader.LeaderLatch$9.apply(LeaderLatch.java:640)
        at org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:92)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)

I assume this is where the failure occurs. This seems to end the thread, possibly without freeing the thread's resources -- I think this because Chronos remains connected and registered to Mesos and continues to receive (but neither accept nor decline -- which causes Mesos work to completely halt) resource offers. Chronos continues to support its web frontend and REST API as well, but performs no further work with jobs, tasks or Mesos.

FxChiP commented 9 years ago

It looks like the key may be the leader election process:

[2014-10-28 01:54:36,231] INFO State change: SUSPENDED (org.apache.curator.framework.state.ConnectionStateManager:228)
[2014-10-28 01:54:36,360] INFO Defeated. Not the current leader. (com.airbnb.scheduler.jobs.JobScheduler:665)
[2014-10-28 01:54:37,073] INFO Opening socket connection to server <ZK-1-SERVER>. Will not attempt to authenticate using SASL (unknown error) (org.apache.zookeeper.ClientCnxn:975)
[2014-10-28 01:54:37,075] INFO Socket connection established to <ZK-1-SERVER>, initiating session (org.apache.zookeeper.ClientCnxn:852)
[2014-10-28 01:54:37,080] WARN Connected to an old server; r-o mode will be unavailable (org.apache.zookeeper.ClientCnxnSocket:139)
[2014-10-28 01:54:37,080] INFO Session establishment complete on server <ZK-1-SERVER>, sessionid = 0x249167f29af001f, negotiated timeout = 40000 (org.apache.zookeeper.ClientCnxn:1235)
[2014-10-28 01:54:37,080] INFO State change: RECONNECTED (org.apache.curator.framework.state.ConnectionStateManager:228)
[2014-10-28 01:54:37,160] INFO Elected as leader. (com.airbnb.scheduler.jobs.JobScheduler:671)

Between onDefeated and onElected, the JobScheduler is not destroyed or regenerated, and the jobGraph is not cleared or otherwise reset, destroyed or recreated in any way. So when onElected calls JobUtils.loadJobs, JobUtils.loadJobs calls JobScheduler.registerJob, and JobScheduler.registerJob puts a vertex into the graph that already exists.

What's odd is that Chronos remained connected and registered to Mesos, even though onDefeated called mesosDriver.close()... looking into this next.

FxChiP commented 9 years ago

So mesosDriver.close() calls into MesosSchedulerDriverFactory.close(), which closes out the current MesosSchedulerDriver using MesosSchedulerDriver.stop(). MesosSchedulerDriver.stop() enqueues a "stop" call into the SchedulerProcess process, and when that "stop" call is run, the SchedulerProcess is destroyed -- HOWEVER, when destroying a process, libprocess will not destroy any permanent links created by that process. Instead, the link is maintained until another process claims it (i.e. by requesting the same link). This causes the Mesos server to hold the resource offers for the framework until re-registration -- it will not rescind them, and there doesn't seem to be a timeout for whatever reason. Re-registration never happens because of the exception generated by re-adding a vertex that exists -- the driver is never re-started, so Chronos effectively loses all ability to schedule and Mesos, for not timing out or rescinding resource offers, hangs all other work waiting for a resource accept/decline that cannot occur.

FxChiP commented 9 years ago

This patch is a hack, but it resolves the issue for me:

diff --git a/src/main/scala/com/airbnb/scheduler/jobs/JobScheduler.scala b/src/m
index 426f113..ca6f9d3 100644
--- a/src/main/scala/com/airbnb/scheduler/jobs/JobScheduler.scala
+++ b/src/main/scala/com/airbnb/scheduler/jobs/JobScheduler.scala
@@ -666,6 +666,7 @@ class JobScheduler @Inject()(val scheduleHorizon: Period,

     log.info("Defeated. Not the current leader.")
     running.set(false)
+    jobGraph.reset() // So we can rebuild it later.
     schedulerThreadFuture.get.cancel(true)
   }
FxChiP commented 9 years ago

Steps to reproduce, for anyone interested:

1.) On the server running the Chronos scheduler, run this command to create an iptables rule that will drop all outbound zookeeper connections:

iptables -I OUTPUT -p tcp --dport 2181 -j REJECT --reject-with tcp-reset -m state --state NEW,ESTABLISHED

2.) Wait a second and then check the Chronos log for an entry starting with "Defeated." This indicates that the Chronos scheduler has lost its leadership of the cluster. Once it has lost its leadership, remove the iptables rule that segments Chronos from ZooKeeper:

iptables -D OUTPUT 1

3.) Allow Chronos to reconnect and check the log for "Elected". The latest entry for this line is the last time it was elected leader. Some log entries afterward, you should see the exception described in the first post. If you have already applied my patch, Chronos should continue working as expected instead.

guyfig commented 9 years ago

Thanks for finding it.

brndnmtthws commented 9 years ago

Could you please submit a PR for the patch?

brndnmtthws commented 9 years ago

Closing as per #300.

bobrik commented 9 years ago

@brndnmtthws can you release version with fix? I'm seeing this bug for the third time today

brndnmtthws commented 9 years ago

We don't publish jars. Mesosphere provides some, so you could ask them.

I'd suggest you just compile the jar yourself, as it's trivial to do.

bobrik commented 9 years ago

@brndnmtthws already asked mesosphere. I'm using docker image from their build, so for me it's better to just update an image.