From 4c6b561eab54920f5d2cbad900042b3a7ecbcb79 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Fri, 19 Dec 2025 17:39:28 +0800 Subject: [PATCH 1/3] [SPARK-54774][CORE] k8s submit failed should keep same exit code with user code --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 6872c7c3bd71..4d1687f091d5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -1047,7 +1047,7 @@ private[spark] class SparkSubmit extends Logging { !isSqlShell(args.mainClass) && !isThriftServer(args.mainClass) && !isConnectServer(args.mainClass)) { try { - SparkContext.getActive.foreach(_.stop()) + SparkContext.getActive.foreach(_.stop(exitCode)) } catch { case e: Throwable => logError("Failed to close SparkContext", e) } From 75a1c74f9ad266def1f98bd5116c467e842588f2 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Tue, 23 Dec 2025 11:39:13 +0800 Subject: [PATCH 2/3] Update SparkSubmitSuite.scala --- .../spark/deploy/SparkSubmitSuite.scala | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 18d3c35ea94f..6302e6c992b4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -1631,6 +1631,22 @@ class SparkSubmitSuite assertResult(3)(runSparkSubmit(args, expectFailure = true)) } + test("SPARK-54774: k8s submit failed should keep same exit code with user code") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", K8sExitCodeTestApplication.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "k8s://host:port", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.kubernetes.authenticate.driver.serviceAccountName=default", + unusedJar.toString + ) + // The test application throws SparkUserAppException with exit code 42, + // so SparkContext.stop(42) should be called in k8s mode + assertResult(42)(runSparkSubmit(args, expectFailure = true)) + } + private def testRemoteResources( enableHttpFs: Boolean, forceDownloadSchemes: Seq[String] = Nil): Unit = { @@ -2038,3 +2054,23 @@ class TestSparkApplication extends SparkApplication with Matchers { } } + +object K8sExitCodeTestApplication { + def main(args: Array[String]): Unit = { + TestUtils.configTestLog4j2("INFO") + // Use local master to ensure SparkContext can be created in test environment + // The k8s master is set in SparkSubmit args, which triggers the finally block logic + val conf = new SparkConf().setMaster("local[2]") + val sc = new SparkContext(conf) + try { + // Create a simple RDD to ensure SparkContext is active + sc.parallelize(1 to 10).count() + // Throw SparkUserAppException with a specific exit code + // This simulates a user application failure + throw new SparkUserAppException(42) + } finally { + // Note: In k8s mode, SparkSubmit should call sc.stop(42) in the finally block + // We don't call stop() here to let SparkSubmit handle it + } + } +} From 606ddf03c927de613d50b5b617f9ab50f88533e7 Mon Sep 17 00:00:00 2001 From: Angerszhuuuu Date: Wed, 24 Dec 2025 10:46:16 +0800 Subject: [PATCH 3/3] Update SparkSubmitSuite.scala --- .../spark/deploy/SparkSubmitSuite.scala | 40 +++++++++++++------ 1 file changed, 28 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 6302e6c992b4..b248dbbbea0a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -1633,18 +1633,34 @@ class SparkSubmitSuite test("SPARK-54774: k8s submit failed should keep same exit code with user code") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) - val args = Seq( - "--class", K8sExitCodeTestApplication.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "k8s://host:port", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", "spark.kubernetes.authenticate.driver.serviceAccountName=default", - unusedJar.toString - ) - // The test application throws SparkUserAppException with exit code 42, - // so SparkContext.stop(42) should be called in k8s mode - assertResult(42)(runSparkSubmit(args, expectFailure = true)) + val fileSystem = Utils.getHadoopFileSystem("/", + SparkHadoopUtil.get.newConfiguration(new SparkConf())) + withTempDir { testDir => + val testDirPath = new Path(testDir.getAbsolutePath()) + val args = Seq( + "--class", K8sExitCodeTestApplication.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "k8s://host:port", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.kubernetes.authenticate.driver.serviceAccountName=default", + "--conf", "spark.eventLog.enabled=true", + "--conf", "spark.eventLog.rolling.enabled=false", + "--conf", "spark.eventLog.testing=true", + "--conf", s"spark.eventLog.dir=${testDirPath.toUri.toString}", + unusedJar.toString + ) + // The test application throws SparkUserAppException with exit code 42, + // so SparkContext.stop(42) should be called in k8s mode + runSparkSubmit(args, expectFailure = true) + val listStatus = fileSystem.listStatus(testDirPath) + val logData = EventLogFileReader.openEventLog(listStatus.last.getPath, fileSystem) + Source.fromInputStream(logData)(Codec.UTF8).getLines().filter { line => + line.contains("SparkListenerApplicationEnd") + }.foreach { line => + assert(line.contains("\"ExitCode\":42")) + } + } } private def testRemoteResources(