Skip to content

Commit c8fb9fa

Browse files
author
Andrew Korzhuev
committed
Allow multiple imagePullSecrets
1 parent 4173977 commit c8fb9fa

File tree

7 files changed

+68
-16
lines changed

7 files changed

+68
-16
lines changed

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -54,9 +54,10 @@ private[spark] object Config extends Logging {
5454
.checkValues(Set("Always", "Never", "IfNotPresent"))
5555
.createWithDefault("IfNotPresent")
5656

57-
val IMAGE_PULL_SECRET =
58-
ConfigBuilder("spark.kubernetes.container.image.pullSecret")
59-
.doc("Specifies the Kubernetes secret used to access private image registry.")
57+
val IMAGE_PULL_SECRETS =
58+
ConfigBuilder("spark.kubernetes.container.image.pullSecrets")
59+
.doc("Comma separated list of the Kubernetes secrets used " +
60+
"to access private image registries.")
6061
.stringConf
6162
.createOptional
6263

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,8 @@
1616
*/
1717
package org.apache.spark.deploy.k8s
1818

19+
import io.fabric8.kubernetes.api.model.LocalObjectReference
20+
1921
import org.apache.spark.SparkConf
2022
import org.apache.spark.util.Utils
2123

@@ -35,6 +37,17 @@ private[spark] object KubernetesUtils {
3537
sparkConf.getAllWithPrefix(prefix).toMap
3638
}
3739

40+
/**
41+
* Parses comma-separated list of imagePullSecrets into K8s-understandable format
42+
*/
43+
def parseImagePullSecrets(imagePullSecrets: Option[String]): List[LocalObjectReference] = {
44+
imagePullSecrets match {
45+
case Some(secretsCommaSeparated) =>
46+
secretsCommaSeparated.split(',').map(_.trim).map(new LocalObjectReference(_)).toList
47+
case None => Nil
48+
}
49+
}
50+
3851
def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = {
3952
opt1.foreach { _ => require(opt2.isEmpty, errMessage) }
4053
}

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStep.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ private[spark] class BasicDriverConfigurationStep(
5151
.get(DRIVER_CONTAINER_IMAGE)
5252
.getOrElse(throw new SparkException("Must specify the driver container image"))
5353

54-
private val imagePullSecret = sparkConf.get(IMAGE_PULL_SECRET)
54+
private val imagePullSecrets = sparkConf.get(IMAGE_PULL_SECRETS)
5555

5656
// CPU settings
5757
private val driverCpuCores = sparkConf.getOption("spark.driver.cores").getOrElse("1")
@@ -134,7 +134,7 @@ private[spark] class BasicDriverConfigurationStep(
134134
case _ => driverContainerWithoutArgs.addToArgs(appArgs: _*).build()
135135
}
136136

137-
val imagePullSecrets = imagePullSecret.map(new LocalObjectReference(_)).toList
137+
val parsedImagePullSecrets = KubernetesUtils.parseImagePullSecrets(imagePullSecrets)
138138

139139
val baseDriverPod = new PodBuilder(driverSpec.driverPod)
140140
.editOrNewMetadata()
@@ -145,7 +145,7 @@ private[spark] class BasicDriverConfigurationStep(
145145
.withNewSpec()
146146
.withRestartPolicy("Never")
147147
.withNodeSelector(nodeSelector.asJava)
148-
.withImagePullSecrets(imagePullSecrets.asJava)
148+
.withImagePullSecrets(parsedImagePullSecrets.asJava)
149149
.endSpec()
150150
.build()
151151

resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -68,7 +68,7 @@ private[spark] class ExecutorPodFactory(
6868
.get(EXECUTOR_CONTAINER_IMAGE)
6969
.getOrElse(throw new SparkException("Must specify the executor container image"))
7070
private val imagePullPolicy = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY)
71-
private val imagePullSecret = sparkConf.get(IMAGE_PULL_SECRET)
71+
private val imagePullSecrets = sparkConf.get(IMAGE_PULL_SECRETS)
7272
private val blockManagerPort = sparkConf
7373
.getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
7474

@@ -99,7 +99,7 @@ private[spark] class ExecutorPodFactory(
9999
nodeToLocalTaskCount: Map[String, Int]): Pod = {
100100
val name = s"$executorPodNamePrefix-exec-$executorId"
101101

102-
val imagePullSecrets = imagePullSecret.map(new LocalObjectReference(_)).toList
102+
val parsedImagePullSecrets = KubernetesUtils.parseImagePullSecrets(imagePullSecrets)
103103

104104
// hostname must be no longer than 63 characters, so take the last 63 characters of the pod
105105
// name as the hostname. This preserves uniqueness since the end of name contains
@@ -196,7 +196,7 @@ private[spark] class ExecutorPodFactory(
196196
.withHostname(hostname)
197197
.withRestartPolicy("Never")
198198
.withNodeSelector(nodeSelector.asJava)
199-
.withImagePullSecrets(imagePullSecrets.asJava)
199+
.withImagePullSecrets(parsedImagePullSecrets.asJava)
200200
.endSpec()
201201
.build()
202202

Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
package org.apache.spark.deploy.k8s
18+
19+
import io.fabric8.kubernetes.api.model.LocalObjectReference
20+
21+
import org.apache.spark.SparkFunSuite
22+
23+
class KubernetesUtilsTest extends SparkFunSuite {
24+
25+
test("testParseImagePullSecrets") {
26+
val noSecrets = KubernetesUtils.parseImagePullSecrets(None)
27+
assert(noSecrets === Nil)
28+
29+
val oneSecret = KubernetesUtils.parseImagePullSecrets(Some("imagePullSecret"))
30+
assert(oneSecret === new LocalObjectReference("imagePullSecret") :: Nil)
31+
32+
val commaSeparatedSecrets = KubernetesUtils.parseImagePullSecrets(Some("s1, s2 , s3,s4"))
33+
assert(commaSeparatedSecrets.map(_.getName) === "s1" :: "s2" :: "s3" :: "s4" :: Nil)
34+
}
35+
36+
}

resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/steps/BasicDriverConfigurationStepSuite.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ class BasicDriverConfigurationStepSuite extends SparkFunSuite {
5151
.set(s"$KUBERNETES_DRIVER_ANNOTATION_PREFIX$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE)
5252
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1")
5353
.set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2")
54-
.set(IMAGE_PULL_SECRET, "imagePullSecret")
54+
.set(IMAGE_PULL_SECRETS, "imagePullSecret1, imagePullSecret2")
5555

5656
val submissionStep = new BasicDriverConfigurationStep(
5757
APP_ID,
@@ -107,8 +107,9 @@ class BasicDriverConfigurationStepSuite extends SparkFunSuite {
107107

108108
val driverPodSpec = preparedDriverSpec.driverPod.getSpec
109109
assert(driverPodSpec.getRestartPolicy === "Never")
110-
assert(driverPodSpec.getImagePullSecrets.size() === 1)
111-
assert(driverPodSpec.getImagePullSecrets.get(0).getName === "imagePullSecret")
110+
assert(driverPodSpec.getImagePullSecrets.size() === 2)
111+
assert(driverPodSpec.getImagePullSecrets.get(0).getName === "imagePullSecret1")
112+
assert(driverPodSpec.getImagePullSecrets.get(1).getName === "imagePullSecret2")
112113

113114
val resolvedSparkConf = preparedDriverSpec.driverSparkConf.getAll.toMap
114115
val expectedSparkConf = Map(

resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
3333
private val driverPodUid: String = "driver-uid"
3434
private val executorPrefix: String = "base"
3535
private val executorImage: String = "executor-image"
36-
private val imagePullSecret: String = "imagePullSecret"
36+
private val imagePullSecrets: String = "imagePullSecret1, imagePullSecret2"
3737
private val driverPod = new PodBuilder()
3838
.withNewMetadata()
3939
.withName(driverPodName)
@@ -55,7 +55,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
5555
.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix)
5656
.set(CONTAINER_IMAGE, executorImage)
5757
.set(KUBERNETES_DRIVER_SUBMIT_CHECK, true)
58-
.set(IMAGE_PULL_SECRET, imagePullSecret)
58+
.set(IMAGE_PULL_SECRETS, imagePullSecrets)
5959
}
6060

6161
test("basic executor pod has reasonable defaults") {
@@ -76,8 +76,9 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef
7676
assert(executor.getSpec.getContainers.get(0).getResources.getLimits.size() === 1)
7777
assert(executor.getSpec.getContainers.get(0).getResources
7878
.getLimits.get("memory").getAmount === "1408Mi")
79-
assert(executor.getSpec.getImagePullSecrets.size() === 1)
80-
assert(executor.getSpec.getImagePullSecrets.get(0).getName === imagePullSecret)
79+
assert(executor.getSpec.getImagePullSecrets.size() === 2)
80+
assert(executor.getSpec.getImagePullSecrets.get(0).getName === "imagePullSecret1")
81+
assert(executor.getSpec.getImagePullSecrets.get(1).getName === "imagePullSecret2")
8182

8283
// The pod has no node selector, volumes.
8384
assert(executor.getSpec.getNodeSelector.isEmpty)

0 commit comments

Comments
 (0)