@@ -1631,6 +1631,22 @@ class SparkSubmitSuite
16311631 assertResult(3 )(runSparkSubmit(args, expectFailure = true ))
16321632 }
16331633
1634+ test(" SPARK-54774: k8s submit failed should keep same exit code with user code" ) {
1635+ val unusedJar = TestUtils .createJarWithClasses(Seq .empty)
1636+ val args = Seq (
1637+ " --class" , K8sExitCodeTestApplication .getClass.getName.stripSuffix(" $" ),
1638+ " --name" , " testApp" ,
1639+ " --master" , " k8s://host:port" ,
1640+ " --conf" , " spark.ui.enabled=false" ,
1641+ " --conf" , " spark.master.rest.enabled=false" ,
1642+ " --conf" , " spark.kubernetes.authenticate.driver.serviceAccountName=default" ,
1643+ unusedJar.toString
1644+ )
1645+ // The test application throws SparkUserAppException with exit code 42,
1646+ // so SparkContext.stop(42) should be called in k8s mode
1647+ assertResult(42 )(runSparkSubmit(args, expectFailure = true ))
1648+ }
1649+
16341650 private def testRemoteResources (
16351651 enableHttpFs : Boolean ,
16361652 forceDownloadSchemes : Seq [String ] = Nil ): Unit = {
@@ -2038,3 +2054,23 @@ class TestSparkApplication extends SparkApplication with Matchers {
20382054 }
20392055
20402056}
2057+
2058+ object K8sExitCodeTestApplication {
2059+ def main (args : Array [String ]): Unit = {
2060+ TestUtils .configTestLog4j2(" INFO" )
2061+ // Use local master to ensure SparkContext can be created in test environment
2062+ // The k8s master is set in SparkSubmit args, which triggers the finally block logic
2063+ val conf = new SparkConf ().setMaster(" local[2]" )
2064+ val sc = new SparkContext (conf)
2065+ try {
2066+ // Create a simple RDD to ensure SparkContext is active
2067+ sc.parallelize(1 to 10 ).count()
2068+ // Throw SparkUserAppException with a specific exit code
2069+ // This simulates a user application failure
2070+ throw new SparkUserAppException (42 )
2071+ } finally {
2072+ // Note: In k8s mode, SparkSubmit should call sc.stop(42) in the finally block
2073+ // We don't call stop() here to let SparkSubmit handle it
2074+ }
2075+ }
2076+ }
0 commit comments