Skip to content

Commit

Permalink
[SPARK-3736] Workers reconnect when disassociated from the master.
Browse files Browse the repository at this point in the history
Before, if the master node is killed and restarted, the worker nodes
would not attempt to reconnect to the Master. Therefore, when the Master
node was restarted, the worker nodes needed to be restarted as well.

Now, when the Master node is disconnected, the worker nodes will
continuously ping the master node in attempts to reconnect to it. Once
the master node restarts, it will detect one of the registration
requests from its former workers. The result is that the cluster
re-enters a healthy state.

In addition, when the master does not receive a heartbeat from the
worker, the worker was removed; however, when the worker sent a
heartbeat to the master, the master used to ignore the heartbeat. Now,
a master that receives a heartbeat from a worker that had been
disconnected will request the worker to re-attempt the registration
process, at which point the worker will send a RegisterWorker request
and be re-connected accordingly.

Re-connection attempts per worker are submitted every N seconds, where N
is configured by the property spark.worker.reconnect.interval - this has
a default of 60 seconds right now.
  • Loading branch information
mccheah committed Oct 15, 2014
1 parent 293a0b5 commit b5b34af
Show file tree
Hide file tree
Showing 3 changed files with 27 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,8 @@ private[deploy] object DeployMessages {

case class RegisterWorkerFailed(message: String) extends DeployMessage

case class ReconnectWorker(masterUrl: String) extends DeployMessage

case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage

case class LaunchExecutor(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -341,7 +341,11 @@ private[spark] class Master(
case Some(workerInfo) =>
workerInfo.lastHeartbeat = System.currentTimeMillis()
case None =>
logWarning("Got heartbeat from unregistered worker " + workerId)
if (workers.map(_.id).contains(workerId)) {
logWarning(s"Got heartbeat from unregistered worker $workerId." +
" Asking it to re-register.")
sender ! ReconnectWorker(masterUrl)
}
}
}

Expand Down
20 changes: 20 additions & 0 deletions core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,8 @@ private[spark] class Worker(
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3

val RECONNECT_ATTEMPT_INTERVAL_MILLIS = conf.getLong("spark.worker.reconnect.interval", 60) * 1000

val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false)
// How often worker will clean up old app folders
val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000
Expand Down Expand Up @@ -94,6 +96,7 @@ private[spark] class Worker(
val finishedExecutors = new HashMap[String, ExecutorRunner]
val drivers = new HashMap[String, DriverRunner]
val finishedDrivers = new HashMap[String, DriverRunner]
var scheduledReconnectMessage: Option[Cancellable] = None

val publicAddress = {
val envVar = System.getenv("SPARK_PUBLIC_DNS")
Expand Down Expand Up @@ -197,6 +200,8 @@ private[spark] class Worker(
context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis,
CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup)
}
scheduledReconnectMessage.foreach(_.cancel())
scheduledReconnectMessage = None

case SendHeartbeat =>
if (connected) { master ! Heartbeat(workerId) }
Expand Down Expand Up @@ -243,6 +248,10 @@ private[spark] class Worker(
System.exit(1)
}

case ReconnectWorker(masterUrl) =>
logWarning(s"Master with url $masterUrl requested this worker to reconnect.")
scheduleAttemptsToReconnectToMaster()

case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_) =>
if (masterUrl != activeMasterUrl) {
logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.")
Expand Down Expand Up @@ -365,6 +374,16 @@ private[spark] class Worker(
def masterDisconnected() {
logError("Connection to master failed! Waiting for master to reconnect...")
connected = false
scheduleAttemptsToReconnectToMaster()
}

def scheduleAttemptsToReconnectToMaster() {
if (!scheduledReconnectMessage.isDefined) {
scheduledReconnectMessage = Some(context.system.scheduler.schedule(
Duration Zero, RECONNECT_ATTEMPT_INTERVAL_MILLIS millis) {
tryRegisterAllMasters()
})
}
}

def generateWorkerId(): String = {
Expand All @@ -374,6 +393,7 @@ private[spark] class Worker(
override def postStop() {
metricsSystem.report()
registrationRetryTimer.foreach(_.cancel())
scheduledReconnectMessage.foreach(_.cancel())
executors.values.foreach(_.kill())
drivers.values.foreach(_.kill())
webUi.stop()
Expand Down

0 comments on commit b5b34af

Please sign in to comment.