@@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster
1919
2020import org .apache .hadoop .yarn .api .records .{ApplicationId , YarnApplicationState }
2121import org .apache .spark .{SparkException , Logging , SparkContext }
22- import org .apache .spark .deploy .yarn .{Client , ClientArguments , YarnSparkHadoopUtil }
22+ import org .apache .spark .deploy .yarn .{Client , ClientArguments }
2323import org .apache .spark .scheduler .TaskSchedulerImpl
2424
2525import scala .collection .mutable .ArrayBuffer
@@ -36,7 +36,6 @@ private[spark] class YarnClientSchedulerBackend(
3636
3737 var client : Client = null
3838 var appId : ApplicationId = null
39- var checkerThread : Thread = null
4039 var stopping : Boolean = false
4140 var totalExpectedExecutors = 0
4241
@@ -83,66 +82,62 @@ private[spark] class YarnClientSchedulerBackend(
8382 totalExpectedExecutors = args.numExecutors
8483 client = new Client (args, conf)
8584 appId = client.submitApplication()
86- waitForApp ()
87- checkerThread = yarnApplicationStateCheckerThread ()
85+ waitForApplication ()
86+ asyncMonitorApplication ()
8887 }
8988
90- def waitForApp () {
91-
92- // TODO : need a better way to find out whether the executors are ready or not
93- // maybe by resource usage report?
94- while (true ) {
95- val report = client.getApplicationReport(appId)
96-
97- logInfo(" Application report from ASM: \n " +
98- " \t appMasterRpcPort: " + report.getRpcPort() + " \n " +
99- " \t appStartTime: " + report.getStartTime() + " \n " +
100- " \t yarnAppState: " + report.getYarnApplicationState() + " \n "
101- )
102-
103- // Ready to go, or already gone.
104- val state = report.getYarnApplicationState()
105- if (state == YarnApplicationState .RUNNING ) {
106- return
107- } else if (state == YarnApplicationState .FINISHED ||
108- state == YarnApplicationState .FAILED ||
109- state == YarnApplicationState .KILLED ) {
110- throw new SparkException (" Yarn application already ended," +
111- " might be killed or not able to launch application master." )
112- }
113-
114- Thread .sleep(1000 )
89+ /**
90+ * Report the state of the application until it is running.
91+ * If the application has finished, failed or been killed in the process, throw an exception.
92+ * This assumes both `client` and `appId` have already been set.
93+ */
94+ private def waitForApplication (): Unit = {
95+ assert(client != null && appId != null , " Application has not been submitted yet!" )
96+ val state = client.monitorApplication(appId, returnOnRunning = true ) // blocking
97+ if (state == YarnApplicationState .FINISHED ||
98+ state == YarnApplicationState .FAILED ||
99+ state == YarnApplicationState .KILLED ) {
100+ throw new SparkException (" Yarn application has already ended! " +
101+ " It might have been killed or unable to launch application master." )
102+ }
103+ if (state == YarnApplicationState .RUNNING ) {
104+ logInfo(s " Application ${appId.getId} has started running. " )
115105 }
116106 }
117107
118- private def yarnApplicationStateCheckerThread (): Thread = {
108+ /**
109+ * Monitor the application state in a separate thread.
110+ * If the application has exited for any reason, stop the SparkContext.
111+ * This assumes both `client` and `appId` have already been set.
112+ */
113+ private def asyncMonitorApplication (): Thread = {
114+ assert(client != null && appId != null , " Application has not been submitted yet!" )
119115 val t = new Thread {
120116 override def run () {
121- while (! stopping) {
122- val report = client.getApplicationReport(appId)
123- val state = report.getYarnApplicationState()
124- if (state == YarnApplicationState .FINISHED || state == YarnApplicationState .KILLED
125- || state == YarnApplicationState .FAILED ) {
126- logError(s " Yarn application already ended: $state" )
127- sc.stop()
128- stopping = true
129- }
130- Thread .sleep(1000L )
117+ val state = client.monitorApplication(appId, logApplicationReport = false ) // blocking
118+ if (state == YarnApplicationState .FINISHED ||
119+ state == YarnApplicationState .KILLED ||
120+ state == YarnApplicationState .FAILED ) {
121+ logWarning(s " Yarn application has exited: $state" )
122+ sc.stop()
123+ stopping = true
131124 }
132- checkerThread = null
133- Thread .currentThread().interrupt()
134125 }
135126 }
136- t.setName(" Yarn Application State Checker " )
127+ t.setName(" Yarn Application State Monitor " )
137128 t.setDaemon(true )
138129 t.start()
139130 t
140131 }
141132
133+ /**
134+ * Stop the scheduler. This assumes `start()` has already been called.
135+ */
142136 override def stop () {
137+ assert(client != null , " Attempted to stop this scheduler before starting it!" )
143138 stopping = true
144139 super .stop()
145- client.stop
140+ client.stop()
146141 logInfo(" Stopped" )
147142 }
148143
0 commit comments