Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit a4bc800

Browse files
ash211mccheah
authored andcommitted
Set the REST service's exit code to the exit code of its driver subprocess (#138)
1 parent 25a209b commit a4bc800

File tree

1 file changed

+7
-1
lines changed

1 file changed

+7
-1
lines changed

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ import java.io.{File, FileOutputStream, StringReader}
2020
import java.net.URI
2121
import java.nio.file.Paths
2222
import java.util.concurrent.CountDownLatch
23+
import java.util.concurrent.atomic.AtomicInteger
2324
import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
2425

2526
import com.google.common.base.Charsets
@@ -101,6 +102,7 @@ private[spark] class KubernetesSparkRestServer(
101102
conf: SparkConf,
102103
expectedApplicationSecret: Array[Byte],
103104
shutdownLock: CountDownLatch,
105+
exitCode: AtomicInteger,
104106
sslOptions: SSLOptions = new SSLOptions)
105107
extends RestSubmissionServer(host, port, conf, sslOptions) {
106108

@@ -238,7 +240,8 @@ private[spark] class KubernetesSparkRestServer(
238240
})
239241
waitForProcessCompleteExecutor.submit(new Runnable {
240242
override def run(): Unit = {
241-
process.waitFor
243+
// set the REST service's exit code to the exit code of the driver subprocess
244+
exitCode.set(process.waitFor)
242245
SERVLET_LOCK.synchronized {
243246
logInfo("Spark application complete. Shutting down submission server...")
244247
KubernetesSparkRestServer.this.stop
@@ -355,12 +358,14 @@ private[spark] object KubernetesSparkRestServer {
355358
}
356359
val secretBytes = Files.toByteArray(secretFile)
357360
val sparkConf = new SparkConf(true)
361+
val exitCode = new AtomicInteger(0)
358362
val server = new KubernetesSparkRestServer(
359363
parsedArguments.host.get,
360364
parsedArguments.port.get,
361365
sparkConf,
362366
secretBytes,
363367
barrier,
368+
exitCode,
364369
sslOptions)
365370
server.start()
366371
ShutdownHookManager.addShutdownHook(() => {
@@ -371,6 +376,7 @@ private[spark] object KubernetesSparkRestServer {
371376
}
372377
})
373378
barrier.await()
379+
System.exit(exitCode.get())
374380
}
375381
}
376382

0 commit comments

Comments
 (0)