[ https://issues.apache.org/jira/browse/SPARK-19264?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15828402#comment-15828402 ]
hustfxj commented on SPARK-19264: --------------------------------- @Sean Owen Why not solve it like AM of Yarn. I remember the applicationMaster monitors the user's program main thread, If the main thread quit, then the AM&Driver will finish. As follows: {code:title=ApplicationMaster.scala|borderStyle=solid} private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. logInfo("Waiting for spark context initialization...") val totalWaitTime = sparkConf.get(AM_MAX_WAIT_TIME) try { val sc = ThreadUtils.awaitResult(sparkContextPromise.future, Duration(totalWaitTime, TimeUnit.MILLISECONDS)) if (sc != null) { rpcEnv = sc.env.rpcEnv val driverRef = runAMEndpoint( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port"), isClusterMode = true) registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl).getOrElse(""), securityMgr) } else { // Sanity check; should never happen in normal operation, since sc should only be null // if the user app did not create a SparkContext. if (!finished) { throw new IllegalStateException("SparkContext is null but app is still running!") } } userClassThread.join() } catch { case e: SparkException if e.getCause().isInstanceOf[TimeoutException] => logError( s"SparkContext did not initialize after waiting for $totalWaitTime ms. " + "Please check earlier log output for errors. Failing the application.") finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_SC_NOT_INITED, "Timed out waiting for SparkContext.") } } {code} > Work should start driver, the same to AM of yarn > --------------------------------------------------- > > Key: SPARK-19264 > URL: https://issues.apache.org/jira/browse/SPARK-19264 > Project: Spark > Issue Type: Improvement > Reporter: hustfxj > > I think work can't start driver by "ProcessBuilderLike", thus we can't > know the application's main thread is finished or not if the application's > main thread contains some non-daemon threads. Because the program terminates > when there no longer is any non-daemon thread running (or someone called > System.exit). The main thread can have finished long ago. > worker should start driver like AM of YARN . As followed: > {code:title=ApplicationMaster.scala|borderStyle=solid} > mainMethod.invoke(null, userArgs.toArray) > finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) > logDebug("Done running users class") > {code} > Then the work can monitor the driver's main thread, and know the > application's state. -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org