Skip to content

Commit 4b3fe3a

Browse files
author
Marcelo Vanzin
committed
[SPARK-25815][K8S] Support kerberos in client mode, keytab-based token renewal.
This change hooks up the k8s backed to the updated HadoopDelegationTokenManager, so that delegation tokens are also available in client mode, and keytab-based token renewal is enabled. The change re-works the k8s feature steps related to kerberos so that the driver does all the credential management and provides all the needed information to executors - so nothing needs to be added to executor pods. This also makes cluster mode behave a lot more similarly to client mode, since no driver-related config steps are run in the latter case. The main two things that don't need to happen in executors anymore are: - adding the Hadoop config to the executor pods: this is not needed since the Spark driver will serialize the Hadoop config and send it to executors when running tasks. - mounting the kerberos config file in the executor pods: this is not needed once you remove the above. The Hadoop conf sent by the driver with the tasks is already resolved (i.e. has all the kerberos names properly defined), so executors do not need access to the kerberos realm information anymore. The change also avoids creating delegation tokens unnecessarily. This means that they'll only be created if a secret with tokens was not provided, and if a keytab is not provided. In either of those cases, the driver code will handle delegation tokens: in cluster mode by creating a secret and stashing them, in client mode by using existing mechanisms to send DTs to executors. One last feature: the change also allows defining a keytab with a "local:" URI. This is supported in client mode (although that's the same as not saying "local:"), and in k8s cluster mode. This allows the keytab to be mounted onto the image from a pre-existing secret, for example. Finally, the new code always sets SPARK_USER in the driver and executor pods. This is in line with how other resource managers behave: the submitting user reflects which user will access Hadoop services in the app. (With kerberos, that's overridden by the logged in user.) That user is unrelated to the OS user the app is running as inside the containers. Tested: - client and cluster mode with kinit - cluster mode with keytab - cluster mode with local: keytab - YARN cluster with keytab (to make sure it isn't broken) Closes #22911 from vanzin/SPARK-25815. Authored-by: Marcelo Vanzin <[email protected]> Signed-off-by: Marcelo Vanzin <[email protected]>
1 parent 428eb2a commit 4b3fe3a

25 files changed

+649
-621
lines changed

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 15 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.deploy
1919

2020
import java.io._
2121
import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException}
22-
import java.net.URL
22+
import java.net.{URI, URL}
2323
import java.security.PrivilegedExceptionAction
2424
import java.text.ParseException
2525
import java.util.UUID
@@ -334,19 +334,20 @@ private[spark] class SparkSubmit extends Logging {
334334
val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf))
335335
val targetDir = Utils.createTempDir()
336336

337-
// assure a keytab is available from any place in a JVM
338-
if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) {
339-
if (args.principal != null) {
340-
if (args.keytab != null) {
341-
require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist")
342-
// Add keytab and principal configurations in sysProps to make them available
343-
// for later use; e.g. in spark sql, the isolated class loader used to talk
344-
// to HiveMetastore will use these settings. They will be set as Java system
345-
// properties and then loaded by SparkConf
346-
sparkConf.set(KEYTAB, args.keytab)
347-
sparkConf.set(PRINCIPAL, args.principal)
348-
UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
349-
}
337+
// Kerberos is not supported in standalone mode, and keytab support is not yet available
338+
// in Mesos cluster mode.
339+
if (clusterManager != STANDALONE
340+
&& !isMesosCluster
341+
&& args.principal != null
342+
&& args.keytab != null) {
343+
// If client mode, make sure the keytab is just a local path.
344+
if (deployMode == CLIENT && Utils.isLocalUri(args.keytab)) {
345+
args.keytab = new URI(args.keytab).getPath()
346+
}
347+
348+
if (!Utils.isLocalUri(args.keytab)) {
349+
require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist")
350+
UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab)
350351
}
351352
}
352353

core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
package org.apache.spark.deploy.security
1919

2020
import java.io.File
21+
import java.net.URI
2122
import java.security.PrivilegedExceptionAction
2223
import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
2324
import java.util.concurrent.atomic.AtomicReference
@@ -71,11 +72,13 @@ private[spark] class HadoopDelegationTokenManager(
7172
private val providerEnabledConfig = "spark.security.credentials.%s.enabled"
7273

7374
private val principal = sparkConf.get(PRINCIPAL).orNull
74-
private val keytab = sparkConf.get(KEYTAB).orNull
75+
76+
// The keytab can be a local: URI for cluster mode, so translate it to a regular path. If it is
77+
// needed later on, the code will check that it exists.
78+
private val keytab = sparkConf.get(KEYTAB).map { uri => new URI(uri).getPath() }.orNull
7579

7680
require((principal == null) == (keytab == null),
7781
"Both principal and keytab must be defined, or neither.")
78-
require(keytab == null || new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
7982

8083
private val delegationTokenProviders = loadProviders()
8184
logDebug("Using the following builtin delegation token providers: " +
@@ -264,6 +267,7 @@ private[spark] class HadoopDelegationTokenManager(
264267

265268
private def doLogin(): UserGroupInformation = {
266269
logInfo(s"Attempting to login to KDC using principal: $principal")
270+
require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.")
267271
val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
268272
logInfo("Successfully logged into KDC.")
269273
ugi

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -92,6 +92,9 @@ private[spark] object Utils extends Logging {
9292
private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
9393
@volatile private var localRootDirs: Array[String] = null
9494

95+
/** Scheme used for files that are locally available on worker nodes in the cluster. */
96+
val LOCAL_SCHEME = "local"
97+
9598
/** Serialize an object using Java serialization */
9699
def serialize[T](o: T): Array[Byte] = {
97100
val bos = new ByteArrayOutputStream()
@@ -2829,6 +2832,11 @@ private[spark] object Utils extends Logging {
28292832
def isClientMode(conf: SparkConf): Boolean = {
28302833
"client".equals(conf.get(SparkLauncher.DEPLOY_MODE, "client"))
28312834
}
2835+
2836+
/** Returns whether the URI is a "local:" URI. */
2837+
def isLocalUri(uri: String): Boolean = {
2838+
uri.startsWith(s"$LOCAL_SCHEME:")
2839+
}
28322840
}
28332841

28342842
private[util] object CallerContext extends Logging {

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

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -87,25 +87,22 @@ private[spark] object Constants {
8787
val NON_JVM_MEMORY_OVERHEAD_FACTOR = 0.4d
8888

8989
// Hadoop Configuration
90-
val HADOOP_FILE_VOLUME = "hadoop-properties"
90+
val HADOOP_CONF_VOLUME = "hadoop-properties"
9191
val KRB_FILE_VOLUME = "krb5-file"
9292
val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf"
9393
val KRB_FILE_DIR_PATH = "/etc"
9494
val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR"
9595
val HADOOP_CONFIG_MAP_NAME =
9696
"spark.kubernetes.executor.hadoopConfigMapName"
97-
val KRB5_CONFIG_MAP_NAME =
98-
"spark.kubernetes.executor.krb5ConfigMapName"
9997

10098
// Kerberos Configuration
101-
val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens"
10299
val KERBEROS_DT_SECRET_NAME =
103100
"spark.kubernetes.kerberos.dt-secret-name"
104101
val KERBEROS_DT_SECRET_KEY =
105102
"spark.kubernetes.kerberos.dt-secret-key"
106-
val KERBEROS_SPARK_USER_NAME =
107-
"spark.kubernetes.kerberos.spark-user-name"
108103
val KERBEROS_SECRET_KEY = "hadoop-tokens"
104+
val KERBEROS_KEYTAB_VOLUME = "kerberos-keytab"
105+
val KERBEROS_KEYTAB_MOUNT_POINT = "/mnt/secrets/kerberos-keytab"
109106

110107
// Hadoop credentials secrets for the Spark app.
111108
val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials"

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

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -42,10 +42,6 @@ private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) {
4242

4343
def appName: String = get("spark.app.name", "spark")
4444

45-
def hadoopConfigMapName: String = s"$resourceNamePrefix-hadoop-config"
46-
47-
def krbConfigMapName: String = s"$resourceNamePrefix-krb5-file"
48-
4945
def namespace: String = get(KUBERNETES_NAMESPACE)
5046

5147
def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY)

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

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,30 @@ package org.apache.spark.deploy.k8s
1818

1919
import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}
2020

21-
private[spark] case class SparkPod(pod: Pod, container: Container)
21+
private[spark] case class SparkPod(pod: Pod, container: Container) {
22+
23+
/**
24+
* Convenience method to apply a series of chained transformations to a pod.
25+
*
26+
* Use it like:
27+
*
28+
* original.modify { case pod =>
29+
* // update pod and return new one
30+
* }.modify { case pod =>
31+
* // more changes that create a new pod
32+
* }.modify {
33+
* case pod if someCondition => // new pod
34+
* }
35+
*
36+
* This makes it cleaner to apply multiple transformations, avoiding having to create
37+
* a bunch of awkwardly-named local variables. Since the argument is a partial function,
38+
* it can do matching without needing to exhaust all the possibilities. If the function
39+
* is not applied, then the original pod will be kept.
40+
*/
41+
def transform(fn: PartialFunction[SparkPod, SparkPod]): SparkPod = fn.lift(this).getOrElse(this)
42+
43+
}
44+
2245

2346
private[spark] object SparkPod {
2447
def initialPod(): SparkPod = {

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,10 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf)
110110
.withContainerPort(driverUIPort)
111111
.withProtocol("TCP")
112112
.endPort()
113+
.addNewEnv()
114+
.withName(ENV_SPARK_USER)
115+
.withValue(Utils.getCurrentUserName())
116+
.endEnv()
113117
.addAllToEnv(driverCustomEnvs.asJava)
114118
.addNewEnv()
115119
.withName(ENV_DRIVER_BIND_ADDRESS)

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

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,10 @@ private[spark] class BasicExecutorFeatureStep(
163163
.addToLimits("memory", executorMemoryQuantity)
164164
.addToRequests("cpu", executorCpuQuantity)
165165
.endResources()
166+
.addNewEnv()
167+
.withName(ENV_SPARK_USER)
168+
.withValue(Utils.getCurrentUserName())
169+
.endEnv()
166170
.addAllToEnv(executorEnv.asJava)
167171
.withPorts(requiredPorts.asJava)
168172
.addToArgs("executor")
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,124 @@
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.features
18+
19+
import java.io.File
20+
import java.nio.charset.StandardCharsets
21+
22+
import scala.collection.JavaConverters._
23+
24+
import com.google.common.io.Files
25+
import io.fabric8.kubernetes.api.model._
26+
27+
import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod}
28+
import org.apache.spark.deploy.k8s.Config._
29+
import org.apache.spark.deploy.k8s.Constants._
30+
31+
/**
32+
* Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration
33+
* directory - on the driver pod.
34+
*/
35+
private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf)
36+
extends KubernetesFeatureConfigStep {
37+
38+
private val confDir = Option(conf.sparkConf.getenv(ENV_HADOOP_CONF_DIR))
39+
private val existingConfMap = conf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP)
40+
41+
KubernetesUtils.requireNandDefined(
42+
confDir,
43+
existingConfMap,
44+
"Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " +
45+
"as the creation of an additional ConfigMap, when one is already specified is extraneous")
46+
47+
private lazy val confFiles: Seq[File] = {
48+
val dir = new File(confDir.get)
49+
if (dir.isDirectory) {
50+
dir.listFiles.filter(_.isFile).toSeq
51+
} else {
52+
Nil
53+
}
54+
}
55+
56+
private def newConfigMapName: String = s"${conf.resourceNamePrefix}-hadoop-config"
57+
58+
private def hasHadoopConf: Boolean = confDir.isDefined || existingConfMap.isDefined
59+
60+
override def configurePod(original: SparkPod): SparkPod = {
61+
original.transform { case pod if hasHadoopConf =>
62+
val confVolume = if (confDir.isDefined) {
63+
val keyPaths = confFiles.map { file =>
64+
new KeyToPathBuilder()
65+
.withKey(file.getName())
66+
.withPath(file.getName())
67+
.build()
68+
}
69+
new VolumeBuilder()
70+
.withName(HADOOP_CONF_VOLUME)
71+
.withNewConfigMap()
72+
.withName(newConfigMapName)
73+
.withItems(keyPaths.asJava)
74+
.endConfigMap()
75+
.build()
76+
} else {
77+
new VolumeBuilder()
78+
.withName(HADOOP_CONF_VOLUME)
79+
.withNewConfigMap()
80+
.withName(existingConfMap.get)
81+
.endConfigMap()
82+
.build()
83+
}
84+
85+
val podWithConf = new PodBuilder(pod.pod)
86+
.editSpec()
87+
.addNewVolumeLike(confVolume)
88+
.endVolume()
89+
.endSpec()
90+
.build()
91+
92+
val containerWithMount = new ContainerBuilder(pod.container)
93+
.addNewVolumeMount()
94+
.withName(HADOOP_CONF_VOLUME)
95+
.withMountPath(HADOOP_CONF_DIR_PATH)
96+
.endVolumeMount()
97+
.addNewEnv()
98+
.withName(ENV_HADOOP_CONF_DIR)
99+
.withValue(HADOOP_CONF_DIR_PATH)
100+
.endEnv()
101+
.build()
102+
103+
SparkPod(podWithConf, containerWithMount)
104+
}
105+
}
106+
107+
override def getAdditionalKubernetesResources(): Seq[HasMetadata] = {
108+
if (confDir.isDefined) {
109+
val fileMap = confFiles.map { file =>
110+
(file.getName(), Files.toString(file, StandardCharsets.UTF_8))
111+
}.toMap.asJava
112+
113+
Seq(new ConfigMapBuilder()
114+
.withNewMetadata()
115+
.withName(newConfigMapName)
116+
.endMetadata()
117+
.addToData(fileMap)
118+
.build())
119+
} else {
120+
Nil
121+
}
122+
}
123+
124+
}

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

Lines changed: 0 additions & 40 deletions
This file was deleted.

0 commit comments

Comments
 (0)