From d6b53c34c5c586fe04e000929412d54383202c0f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 1 Nov 2023 11:53:32 -0700 Subject: [PATCH] [SPARK-45754][CORE] Support `spark.deploy.appIdPattern` ### What changes were proposed in this pull request? This PR aims to support `spark.deploy.appIdPattern` for Apache Spark 4.0.0. ### Why are the changes needed? This allows the users to be able to control driver ID pattern. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs with the newly added test case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43616 from dongjoon-hyun/SPARK-45754. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/deploy/master/Master.scala | 3 ++- .../org/apache/spark/internal/config/Deploy.scala | 9 +++++++++ .../apache/spark/deploy/master/MasterSuite.scala | 15 +++++++++++++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 0a66cc974da7c..058b944c591ad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -54,6 +54,7 @@ private[deploy] class Master( ThreadUtils.newDaemonSingleThreadScheduledExecutor("master-forward-message-thread") private val driverIdPattern = conf.get(DRIVER_ID_PATTERN) + private val appIdPattern = conf.get(APP_ID_PATTERN) // For application IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) @@ -1152,7 +1153,7 @@ private[deploy] class Master( /** Generate a new app ID given an app's submission date */ private def newApplicationId(submitDate: Date): String = { - val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) + val appId = appIdPattern.format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala index bffdc79175bd9..c6ccf9550bc91 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -90,4 +90,13 @@ private[spark] object Deploy { .stringConf .checkValue(!_.format("20231101000000", 0).exists(_.isWhitespace), "Whitespace is not allowed.") .createWithDefault("driver-%s-%04d") + + val APP_ID_PATTERN = ConfigBuilder("spark.deploy.appIdPattern") + .doc("The pattern for app ID generation based on Java `String.format` method.. " + + "The default value is `app-%s-%04d` which represents the existing app id string, " + + "e.g., `app-20231031224509-0008`. Plesae be careful to generate unique IDs.") + .version("4.0.0") + .stringConf + .checkValue(!_.format("20231101000000", 0).exists(_.isWhitespace), "Whitespace is not allowed.") + .createWithDefault("app-%s-%04d") } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index cef0e84f20f7a..e8615cdbdd559 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -803,6 +803,7 @@ class MasterSuite extends SparkFunSuite PrivateMethod[mutable.ArrayBuffer[DriverInfo]](Symbol("waitingDrivers")) private val _state = PrivateMethod[RecoveryState.Value](Symbol("state")) private val _newDriverId = PrivateMethod[String](Symbol("newDriverId")) + private val _newApplicationId = PrivateMethod[String](Symbol("newApplicationId")) private val workerInfo = makeWorkerInfo(4096, 10) private val workerInfos = Array(workerInfo, workerInfo, workerInfo) @@ -1251,6 +1252,20 @@ class MasterSuite extends SparkFunSuite }.getMessage assert(m.contains("Whitespace is not allowed")) } + + test("SPARK-45754: Support app id pattern") { + val master = makeMaster(new SparkConf().set(APP_ID_PATTERN, "my-app-%2$05d")) + val submitDate = new Date() + assert(master.invokePrivate(_newApplicationId(submitDate)) === "my-app-00000") + assert(master.invokePrivate(_newApplicationId(submitDate)) === "my-app-00001") + } + + test("SPARK-45754: Prevent invalid app id patterns") { + val m = intercept[IllegalArgumentException] { + makeMaster(new SparkConf().set(APP_ID_PATTERN, "my app")) + }.getMessage + assert(m.contains("Whitespace is not allowed")) + } } private class FakeRecoveryModeFactory(conf: SparkConf, ser: serializer.Serializer)