Skip to content

Commit 4e8701a

Browse files
committed
[SPARK-35280][K8S] Promote KubernetesUtils to DeveloperApi
### What changes were proposed in this pull request? Since SPARK-22757, `KubernetesUtils` has been used as an important utility class by all K8s modules and `ExternalClusterManager`s. This PR aims to promote `KubernetesUtils` to `DeveloperApi` in order to maintain it officially in a backward compatible way at Apache Spark 3.2.0. ### Why are the changes needed? Apache Spark 3.1.1 makes `Kubernetes` module GA and provides an extensible external cluster manager framework. To have `ExternalClusterManager` for K8s environment, `KubernetesUtils` class is crucial and needs to be stable. By promoting to a subset of K8s developer API, we can maintain these more sustainable way and give a better and stable functionality to K8s users. In this PR, `Since` annotations denote the last function signature changes because these are going to become public at Apache Spark 3.2.0. | Version | Function Name | |-|-| | 2.3.0 | parsePrefixedKeyValuePairs | | 2.3.0 | requireNandDefined | | 2.3.0 | parsePrefixedKeyValuePairs | | 2.4.0 | parseMasterUrl | | 3.0.0 | requireBothOrNeitherDefined | | 3.0.0 | requireSecondIfFirstIsDefined | | 3.0.0 | selectSparkContainer | | 3.0.0 | formatPairsBundle | | 3.0.0 | formatPodState | | 3.0.0 | containersDescription | | 3.0.0 | containerStatusDescription | | 3.0.0 | formatTime | | 3.0.0 | uniqueID | | 3.0.0 | buildResourcesQuantities | | 3.0.0 | uploadAndTransformFileUris | | 3.0.0 | uploadFileUri | | 3.0.0 | requireBothOrNeitherDefined | | 3.0.0 | buildPodWithServiceAccount | | 3.0.0 | isLocalAndResolvable | | 3.1.1 | renameMainAppResource | | 3.1.1 | addOwnerReference | | 3.2.0 | loadPodFromTemplate | ### Does this PR introduce _any_ user-facing change? Yes, but this is new API additions. ### How was this patch tested? Pass the CIs. Closes #32406 from dongjoon-hyun/SPARK-35280. Authored-by: Dongjoon Hyun <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent 39889df commit 4e8701a

File tree

1 file changed

+29
-1
lines changed

1 file changed

+29
-1
lines changed

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala

Lines changed: 29 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.apache.commons.codec.binary.Hex
2929
import org.apache.hadoop.fs.{FileSystem, Path}
3030

3131
import org.apache.spark.{SparkConf, SparkException}
32+
import org.apache.spark.annotation.{DeveloperApi, Since, Unstable}
3233
import org.apache.spark.deploy.SparkHadoopUtil
3334
import org.apache.spark.deploy.k8s.Config.KUBERNETES_FILE_UPLOAD_PATH
3435
import org.apache.spark.internal.Logging
@@ -38,7 +39,14 @@ import org.apache.spark.util.{Clock, SystemClock, Utils}
3839
import org.apache.spark.util.DependencyUtils.downloadFile
3940
import org.apache.spark.util.Utils.getHadoopFileSystem
4041

41-
private[spark] object KubernetesUtils extends Logging {
42+
/**
43+
* :: DeveloperApi ::
44+
*
45+
* A utility class used for K8s operations internally and for implementing ExternalClusterManagers.
46+
*/
47+
@Unstable
48+
@DeveloperApi
49+
object KubernetesUtils extends Logging {
4250

4351
private val systemClock = new SystemClock()
4452
private lazy val RNG = new SecureRandom()
@@ -51,12 +59,14 @@ private[spark] object KubernetesUtils extends Logging {
5159
* @param prefix the given property name prefix
5260
* @return a Map storing the configuration property keys and values
5361
*/
62+
@Since("2.3.0")
5463
def parsePrefixedKeyValuePairs(
5564
sparkConf: SparkConf,
5665
prefix: String): Map[String, String] = {
5766
sparkConf.getAllWithPrefix(prefix).toMap
5867
}
5968

69+
@Since("3.0.0")
6070
def requireBothOrNeitherDefined(
6171
opt1: Option[_],
6272
opt2: Option[_],
@@ -66,6 +76,7 @@ private[spark] object KubernetesUtils extends Logging {
6676
requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing)
6777
}
6878

79+
@Since("3.0.0")
6980
def requireSecondIfFirstIsDefined(
7081
opt1: Option[_],
7182
opt2: Option[_],
@@ -75,11 +86,13 @@ private[spark] object KubernetesUtils extends Logging {
7586
}
7687
}
7788

89+
@Since("2.3.0")
7890
def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = {
7991
opt1.foreach { _ => require(opt2.isEmpty, errMessage) }
8092
opt2.foreach { _ => require(opt1.isEmpty, errMessage) }
8193
}
8294

95+
@Since("3.2.0")
8396
def loadPodFromTemplate(
8497
kubernetesClient: KubernetesClient,
8598
templateFileName: String,
@@ -99,6 +112,7 @@ private[spark] object KubernetesUtils extends Logging {
99112
}
100113
}
101114

115+
@Since("3.0.0")
102116
def selectSparkContainer(pod: Pod, containerName: Option[String]): SparkPod = {
103117
def selectNamedContainer(
104118
containers: List[Container], name: String): Option[(Container, List[Container])] =
@@ -125,8 +139,10 @@ private[spark] object KubernetesUtils extends Logging {
125139
}.getOrElse(SparkPod(pod, new ContainerBuilder().build()))
126140
}
127141

142+
@Since("2.4.0")
128143
def parseMasterUrl(url: String): String = url.substring("k8s://".length)
129144

145+
@Since("3.0.0")
130146
def formatPairsBundle(pairs: Seq[(String, String)], indent: Int = 1) : String = {
131147
// Use more loggable format if value is null or empty
132148
val indentStr = "\t" * indent
@@ -141,6 +157,7 @@ private[spark] object KubernetesUtils extends Logging {
141157
* @param pod Pod
142158
* @return Human readable pod state
143159
*/
160+
@Since("3.0.0")
144161
def formatPodState(pod: Pod): String = {
145162
val details = Seq[(String, String)](
146163
// pod metadata
@@ -164,6 +181,7 @@ private[spark] object KubernetesUtils extends Logging {
164181
formatPairsBundle(details)
165182
}
166183

184+
@Since("3.0.0")
167185
def containersDescription(p: Pod, indent: Int = 1): String = {
168186
p.getStatus.getContainerStatuses.asScala.map { status =>
169187
Seq(
@@ -173,6 +191,7 @@ private[spark] object KubernetesUtils extends Logging {
173191
}.map(p => formatPairsBundle(p, indent)).mkString("\n\n")
174192
}
175193

194+
@Since("3.0.0")
176195
def containerStatusDescription(containerStatus: ContainerStatus)
177196
: Seq[(String, String)] = {
178197
val state = containerStatus.getState
@@ -200,6 +219,7 @@ private[spark] object KubernetesUtils extends Logging {
200219
}.getOrElse(Seq(("container state", "N/A")))
201220
}
202221

222+
@Since("3.0.0")
203223
def formatTime(time: String): String = {
204224
if (time != null) time else "N/A"
205225
}
@@ -212,6 +232,7 @@ private[spark] object KubernetesUtils extends Logging {
212232
* This avoids using a UUID for uniqueness (too long), and relying solely on the current time
213233
* (not unique enough).
214234
*/
235+
@Since("3.0.0")
215236
def uniqueID(clock: Clock = systemClock): String = {
216237
val random = new Array[Byte](3)
217238
synchronized {
@@ -228,6 +249,7 @@ private[spark] object KubernetesUtils extends Logging {
228249
* It assumes we can use the Kubernetes device plugin format: vendor-domain/resource.
229250
* It returns a set with a tuple of vendor-domain/resource and Quantity for each resource.
230251
*/
252+
@Since("3.0.0")
231253
def buildResourcesQuantities(
232254
componentName: String,
233255
sparkConf: SparkConf): Map[String, Quantity] = {
@@ -247,6 +269,7 @@ private[spark] object KubernetesUtils extends Logging {
247269
/**
248270
* Upload files and modify their uris
249271
*/
272+
@Since("3.0.0")
250273
def uploadAndTransformFileUris(fileUris: Iterable[String], conf: Option[SparkConf] = None)
251274
: Iterable[String] = {
252275
fileUris.map { uri =>
@@ -261,11 +284,13 @@ private[spark] object KubernetesUtils extends Logging {
261284
}
262285
}
263286

287+
@Since("3.0.0")
264288
def isLocalAndResolvable(resource: String): Boolean = {
265289
resource != SparkLauncher.NO_RESOURCE &&
266290
isLocalDependency(Utils.resolveURI(resource))
267291
}
268292

293+
@Since("3.1.1")
269294
def renameMainAppResource(
270295
resource: String,
271296
conf: Option[SparkConf] = None,
@@ -281,6 +306,7 @@ private[spark] object KubernetesUtils extends Logging {
281306
}
282307
}
283308

309+
@Since("3.0.0")
284310
def uploadFileUri(uri: String, conf: Option[SparkConf] = None): String = {
285311
conf match {
286312
case Some(sConf) =>
@@ -325,6 +351,7 @@ private[spark] object KubernetesUtils extends Logging {
325351
}
326352
}
327353

354+
@Since("3.0.0")
328355
def buildPodWithServiceAccount(serviceAccount: Option[String], pod: SparkPod): Option[Pod] = {
329356
serviceAccount.map { account =>
330357
new PodBuilder(pod.pod)
@@ -338,6 +365,7 @@ private[spark] object KubernetesUtils extends Logging {
338365

339366
// Add a OwnerReference to the given resources making the pod an owner of them so when
340367
// the pod is deleted, the resources are garbage collected.
368+
@Since("3.1.1")
341369
def addOwnerReference(pod: Pod, resources: Seq[HasMetadata]): Unit = {
342370
if (pod != null) {
343371
val reference = new OwnerReferenceBuilder()

0 commit comments

Comments
 (0)