Skip to content

Commit dbd90e5

Browse files
Marcelo Vanzinmccheah
Marcelo Vanzin
authored andcommitted
[SPARK-26194][K8S] Auto generate auth secret for k8s apps.
This change modifies the logic in the SecurityManager to do two things: - generate unique app secrets also when k8s is being used - only store the secret in the user's UGI on YARN The latter is needed so that k8s won't unnecessarily create k8s secrets for the UGI credentials when only the auth token is stored there. On the k8s side, the secret is propagated to executors using an environment variable instead. This ensures it works in both client and cluster mode. Security doc was updated to mention the feature and clarify that proper access control in k8s should be enabled for it to be secure. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #23174 from vanzin/SPARK-26194.
1 parent b14a26e commit dbd90e5

File tree

13 files changed

+205
-113
lines changed

13 files changed

+205
-113
lines changed

core/src/main/scala/org/apache/spark/SecurityManager.scala

Lines changed: 16 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -348,25 +348,36 @@ private[spark] class SecurityManager(
348348
*/
349349
def initializeAuth(): Unit = {
350350
import SparkMasterRegex._
351+
val k8sRegex = "k8s.*".r
351352

352353
if (!sparkConf.get(NETWORK_AUTH_ENABLED)) {
353354
return
354355
}
355356

357+
// TODO: this really should be abstracted somewhere else.
356358
val master = sparkConf.get(SparkLauncher.SPARK_MASTER, "")
357-
master match {
359+
val storeInUgi = master match {
358360
case "yarn" | "local" | LOCAL_N_REGEX(_) | LOCAL_N_FAILURES_REGEX(_, _) =>
359-
// Secret generation allowed here
361+
true
362+
363+
case k8sRegex() =>
364+
// Don't propagate the secret through the user's credentials in kubernetes. That conflicts
365+
// with the way k8s handles propagation of delegation tokens.
366+
false
367+
360368
case _ =>
361369
require(sparkConf.contains(SPARK_AUTH_SECRET_CONF),
362370
s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.")
363371
return
364372
}
365373

366374
secretKey = Utils.createSecret(sparkConf)
367-
val creds = new Credentials()
368-
creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8))
369-
UserGroupInformation.getCurrentUser().addCredentials(creds)
375+
376+
if (storeInUgi) {
377+
val creds = new Credentials()
378+
creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8))
379+
UserGroupInformation.getCurrentUser().addCredentials(creds)
380+
}
370381
}
371382

372383
// Default SecurityManager only has a single secret key, so ignore appId.

core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala

Lines changed: 36 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -395,15 +395,23 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties {
395395
assert(keyFromEnv === new SecurityManager(conf2).getSecretKey())
396396
}
397397

398-
test("secret key generation") {
399-
Seq(
400-
("yarn", true),
401-
("local", true),
402-
("local[*]", true),
403-
("local[1, 2]", true),
404-
("local-cluster[2, 1, 1024]", false),
405-
("invalid", false)
406-
).foreach { case (master, shouldGenerateSecret) =>
398+
// How is the secret expected to be generated and stored.
399+
object SecretTestType extends Enumeration {
400+
val MANUAL, AUTO, UGI = Value
401+
}
402+
403+
import SecretTestType._
404+
405+
Seq(
406+
("yarn", UGI),
407+
("local", UGI),
408+
("local[*]", UGI),
409+
("local[1, 2]", UGI),
410+
("k8s://127.0.0.1", AUTO),
411+
("local-cluster[2, 1, 1024]", MANUAL),
412+
("invalid", MANUAL)
413+
).foreach { case (master, secretType) =>
414+
test(s"secret key generation: master '$master'") {
407415
val conf = new SparkConf()
408416
.set(NETWORK_AUTH_ENABLED, true)
409417
.set(SparkLauncher.SPARK_MASTER, master)
@@ -412,19 +420,26 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties {
412420
UserGroupInformation.createUserForTesting("authTest", Array()).doAs(
413421
new PrivilegedExceptionAction[Unit]() {
414422
override def run(): Unit = {
415-
if (shouldGenerateSecret) {
416-
mgr.initializeAuth()
417-
val creds = UserGroupInformation.getCurrentUser().getCredentials()
418-
val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY)
419-
assert(secret != null)
420-
assert(new String(secret, UTF_8) === mgr.getSecretKey())
421-
} else {
422-
intercept[IllegalArgumentException] {
423+
secretType match {
424+
case UGI =>
425+
mgr.initializeAuth()
426+
val creds = UserGroupInformation.getCurrentUser().getCredentials()
427+
val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY)
428+
assert(secret != null)
429+
assert(new String(secret, UTF_8) === mgr.getSecretKey())
430+
431+
case AUTO =>
423432
mgr.initializeAuth()
424-
}
425-
intercept[IllegalArgumentException] {
426-
mgr.getSecretKey()
427-
}
433+
val creds = UserGroupInformation.getCurrentUser().getCredentials()
434+
assert(creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) === null)
435+
436+
case MANUAL =>
437+
intercept[IllegalArgumentException] {
438+
mgr.initializeAuth()
439+
}
440+
intercept[IllegalArgumentException] {
441+
mgr.getSecretKey()
442+
}
428443
}
429444
}
430445
}

docs/security.md

Lines changed: 21 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -26,21 +26,29 @@ not documented, Spark does not support.
2626
Spark currently supports authentication for RPC channels using a shared secret. Authentication can
2727
be turned on by setting the `spark.authenticate` configuration parameter.
2828

29-
The exact mechanism used to generate and distribute the shared secret is deployment-specific.
29+
The exact mechanism used to generate and distribute the shared secret is deployment-specific. Unless
30+
specified below, the secret must be defined by setting the `spark.authenticate.secret` config
31+
option. The same secret is shared by all Spark applications and daemons in that case, which limits
32+
the security of these deployments, especially on multi-tenant clusters.
3033

31-
For Spark on [YARN](running-on-yarn.html) and local deployments, Spark will automatically handle
32-
generating and distributing the shared secret. Each application will use a unique shared secret. In
34+
The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should
35+
ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077
36+
respectively by default) are restricted to hosts that are trusted to submit jobs.
37+
38+
### YARN
39+
40+
For Spark on [YARN](running-on-yarn.html), Spark will automatically handle generating and
41+
distributing the shared secret. Each application will use a unique shared secret. In
3342
the case of YARN, this feature relies on YARN RPC encryption being enabled for the distribution of
3443
secrets to be secure.
3544

36-
For other resource managers, `spark.authenticate.secret` must be configured on each of the nodes.
37-
This secret will be shared by all the daemons and applications, so this deployment configuration is
38-
not as secure as the above, especially when considering multi-tenant clusters. In this
39-
configuration, a user with the secret can effectively impersonate any other user.
45+
### Kubernetes
4046

41-
The Rest Submission Server and the MesosClusterDispatcher do not support authentication. You should
42-
ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077
43-
respectively by default) are restricted to hosts that are trusted to submit jobs.
47+
On Kubernetes, Spark will also automatically generate an authentication secret unique to each
48+
application. The secret is propagated to executor pods using environment variables. This means
49+
that any user that can list pods in the namespace where the Spark application is running can
50+
also see their authentication secret. Access control rules should be properly set up by the
51+
Kubernetes admin to ensure that Spark authentication is secure.
4452

4553
<table class="table">
4654
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
@@ -738,10 +746,10 @@ tokens for supported will be created.
738746
## Secure Interaction with Kubernetes
739747

740748
When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens
741-
so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are
742-
shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job:
749+
so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are
750+
shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job:
743751

744-
In all cases you must define the environment variable: `HADOOP_CONF_DIR` or
752+
In all cases you must define the environment variable: `HADOOP_CONF_DIR` or
745753
`spark.kubernetes.hadoop.configMapName.`
746754

747755
It also important to note that the KDC needs to be visible from inside the containers.

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

Lines changed: 57 additions & 39 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ import scala.collection.JavaConverters._
2020

2121
import io.fabric8.kubernetes.api.model._
2222

23-
import org.apache.spark.SparkException
23+
import org.apache.spark.{SecurityManager, SparkConf, SparkException}
2424
import org.apache.spark.deploy.k8s._
2525
import org.apache.spark.deploy.k8s.Config._
2626
import org.apache.spark.deploy.k8s.Constants._
@@ -29,11 +29,12 @@ import org.apache.spark.rpc.RpcEndpointAddress
2929
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
3030
import org.apache.spark.util.Utils
3131

32-
private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutorConf)
32+
private[spark] class BasicExecutorFeatureStep(
33+
kubernetesConf: KubernetesExecutorConf,
34+
secMgr: SecurityManager)
3335
extends KubernetesFeatureConfigStep {
3436

3537
// Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf
36-
private val executorExtraClasspath = kubernetesConf.get(EXECUTOR_CLASS_PATH)
3738
private val executorContainerImage = kubernetesConf
3839
.get(EXECUTOR_CONTAINER_IMAGE)
3940
.getOrElse(throw new SparkException("Must specify the executor container image"))
@@ -87,44 +88,61 @@ private[spark] class BasicExecutorFeatureStep(kubernetesConf: KubernetesExecutor
8788
val executorCpuQuantity = new QuantityBuilder(false)
8889
.withAmount(executorCoresRequest)
8990
.build()
90-
val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
91-
new EnvVarBuilder()
92-
.withName(ENV_CLASSPATH)
93-
.withValue(cp)
94-
.build()
95-
}
96-
val executorExtraJavaOptionsEnv = kubernetesConf
97-
.get(EXECUTOR_JAVA_OPTIONS)
98-
.map { opts =>
99-
val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId,
100-
kubernetesConf.executorId)
101-
val delimitedOpts = Utils.splitCommandString(subsOpts)
102-
delimitedOpts.zipWithIndex.map {
103-
case (opt, index) =>
104-
new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build()
91+
92+
val executorEnv: Seq[EnvVar] = {
93+
(Seq(
94+
(ENV_DRIVER_URL, driverUrl),
95+
(ENV_EXECUTOR_CORES, executorCores.toString),
96+
(ENV_EXECUTOR_MEMORY, executorMemoryString),
97+
(ENV_APPLICATION_ID, kubernetesConf.appId),
98+
// This is to set the SPARK_CONF_DIR to be /opt/spark/conf
99+
(ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL),
100+
(ENV_EXECUTOR_ID, kubernetesConf.executorId)
101+
) ++ kubernetesConf.environment).map { case (k, v) =>
102+
new EnvVarBuilder()
103+
.withName(k)
104+
.withValue(v)
105+
.build()
105106
}
106-
}.getOrElse(Seq.empty[EnvVar])
107-
val executorEnv = (Seq(
108-
(ENV_DRIVER_URL, driverUrl),
109-
(ENV_EXECUTOR_CORES, executorCores.toString),
110-
(ENV_EXECUTOR_MEMORY, executorMemoryString),
111-
(ENV_APPLICATION_ID, kubernetesConf.appId),
112-
// This is to set the SPARK_CONF_DIR to be /opt/spark/conf
113-
(ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL),
114-
(ENV_EXECUTOR_ID, kubernetesConf.executorId)) ++
115-
kubernetesConf.environment)
116-
.map(env => new EnvVarBuilder()
117-
.withName(env._1)
118-
.withValue(env._2)
119-
.build()
120-
) ++ Seq(
121-
new EnvVarBuilder()
122-
.withName(ENV_EXECUTOR_POD_IP)
123-
.withValueFrom(new EnvVarSourceBuilder()
124-
.withNewFieldRef("v1", "status.podIP")
107+
} ++ {
108+
Seq(new EnvVarBuilder()
109+
.withName(ENV_EXECUTOR_POD_IP)
110+
.withValueFrom(new EnvVarSourceBuilder()
111+
.withNewFieldRef("v1", "status.podIP")
112+
.build())
125113
.build())
126-
.build()
127-
) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq
114+
} ++ {
115+
Option(secMgr.getSecretKey()).map { authSecret =>
116+
new EnvVarBuilder()
117+
.withName(SecurityManager.ENV_AUTH_SECRET)
118+
.withValue(authSecret)
119+
.build()
120+
}
121+
} ++ {
122+
kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp =>
123+
new EnvVarBuilder()
124+
.withName(ENV_CLASSPATH)
125+
.withValue(cp)
126+
.build()
127+
}
128+
} ++ {
129+
val userOpts = kubernetesConf.get(EXECUTOR_JAVA_OPTIONS).toSeq.flatMap { opts =>
130+
val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId,
131+
kubernetesConf.executorId)
132+
Utils.splitCommandString(subsOpts)
133+
}
134+
135+
val sparkOpts = Utils.sparkJavaOpts(kubernetesConf.sparkConf,
136+
SparkConf.isExecutorStartupConf)
137+
138+
(userOpts ++ sparkOpts).zipWithIndex.map { case (opt, index) =>
139+
new EnvVarBuilder()
140+
.withName(s"$ENV_JAVA_OPT_PREFIX$index")
141+
.withValue(opt)
142+
.build()
143+
}
144+
}
145+
128146
val requiredPorts = Seq(
129147
(BLOCK_MANAGER_PORT_NAME, blockManagerPort))
130148
.map { case (name, port) =>

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.PodBuilder
2222
import io.fabric8.kubernetes.client.KubernetesClient
2323
import scala.collection.mutable
2424

25-
import org.apache.spark.{SparkConf, SparkException}
25+
import org.apache.spark.{SecurityManager, SparkConf, SparkException}
2626
import org.apache.spark.deploy.k8s.Config._
2727
import org.apache.spark.deploy.k8s.Constants._
2828
import org.apache.spark.deploy.k8s.KubernetesConf
@@ -31,6 +31,7 @@ import org.apache.spark.util.{Clock, Utils}
3131

3232
private[spark] class ExecutorPodsAllocator(
3333
conf: SparkConf,
34+
secMgr: SecurityManager,
3435
executorBuilder: KubernetesExecutorBuilder,
3536
kubernetesClient: KubernetesClient,
3637
snapshotsStore: ExecutorPodsSnapshotsStore,
@@ -135,7 +136,7 @@ private[spark] class ExecutorPodsAllocator(
135136
newExecutorId.toString,
136137
applicationId,
137138
driverPod)
138-
val executorPod = executorBuilder.buildFromFeatures(executorConf)
139+
val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr)
139140
val podWithAttachedContainer = new PodBuilder(executorPod.pod)
140141
.editOrNewSpec()
141142
.addToContainers(executorPod.container)

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

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -94,6 +94,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
9494

9595
val executorPodsAllocator = new ExecutorPodsAllocator(
9696
sc.conf,
97+
sc.env.securityManager,
9798
KubernetesExecutorBuilder(kubernetesClient, sc.conf),
9899
kubernetesClient,
99100
snapshotsStore,
@@ -110,7 +111,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit
110111

111112
new KubernetesClusterSchedulerBackend(
112113
scheduler.asInstanceOf[TaskSchedulerImpl],
113-
sc.env.rpcEnv,
114+
sc,
114115
kubernetesClient,
115116
requestExecutorsService,
116117
snapshotsStore,

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

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import java.util.concurrent.ExecutorService
2121
import io.fabric8.kubernetes.client.KubernetesClient
2222
import scala.concurrent.{ExecutionContext, Future}
2323

24+
import org.apache.spark.SparkContext
2425
import org.apache.spark.deploy.k8s.Config._
2526
import org.apache.spark.deploy.k8s.Constants._
2627
import org.apache.spark.rpc.{RpcAddress, RpcEnv}
@@ -30,15 +31,15 @@ import org.apache.spark.util.{ThreadUtils, Utils}
3031

3132
private[spark] class KubernetesClusterSchedulerBackend(
3233
scheduler: TaskSchedulerImpl,
33-
rpcEnv: RpcEnv,
34+
sc: SparkContext,
3435
kubernetesClient: KubernetesClient,
3536
requestExecutorsService: ExecutorService,
3637
snapshotsStore: ExecutorPodsSnapshotsStore,
3738
podAllocator: ExecutorPodsAllocator,
3839
lifecycleEventHandler: ExecutorPodsLifecycleManager,
3940
watchEvents: ExecutorPodsWatchSnapshotSource,
4041
pollEvents: ExecutorPodsPollingSnapshotSource)
41-
extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) {
42+
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
4243

4344
private implicit val requestExecutorContext = ExecutionContext.fromExecutorService(
4445
requestExecutorsService)

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

Lines changed: 8 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,14 @@ import java.io.File
2020

2121
import io.fabric8.kubernetes.client.KubernetesClient
2222

23-
import org.apache.spark.SparkConf
23+
import org.apache.spark.{SecurityManager, SparkConf}
2424
import org.apache.spark.deploy.k8s._
2525
import org.apache.spark.deploy.k8s.Constants._
2626
import org.apache.spark.deploy.k8s.features._
2727

2828
private[spark] class KubernetesExecutorBuilder(
29-
provideBasicStep: (KubernetesExecutorConf => BasicExecutorFeatureStep) =
30-
new BasicExecutorFeatureStep(_),
29+
provideBasicStep: (KubernetesExecutorConf, SecurityManager) => BasicExecutorFeatureStep =
30+
new BasicExecutorFeatureStep(_, _),
3131
provideSecretsStep: (KubernetesConf => MountSecretsFeatureStep) =
3232
new MountSecretsFeatureStep(_),
3333
provideEnvSecretsStep: (KubernetesConf => EnvSecretsFeatureStep) =
@@ -44,13 +44,16 @@ private[spark] class KubernetesExecutorBuilder(
4444
new HadoopSparkUserExecutorFeatureStep(_),
4545
provideInitialPod: () => SparkPod = () => SparkPod.initialPod()) {
4646

47-
def buildFromFeatures(kubernetesConf: KubernetesExecutorConf): SparkPod = {
47+
def buildFromFeatures(
48+
kubernetesConf: KubernetesExecutorConf,
49+
secMgr: SecurityManager): SparkPod = {
4850
val sparkConf = kubernetesConf.sparkConf
4951
val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME)
5052
val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME)
5153
val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY)
5254

53-
val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf))
55+
val baseFeatures = Seq(provideBasicStep(kubernetesConf, secMgr),
56+
provideLocalDirsStep(kubernetesConf))
5457
val secretFeature = if (kubernetesConf.secretNamesToMountPaths.nonEmpty) {
5558
Seq(provideSecretsStep(kubernetesConf))
5659
} else Nil

0 commit comments

Comments
 (0)