-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-25815][k8s] Support kerberos in client mode, keytab-based token renewal. #22911
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 7 commits
7f86e23
839477f
233ad0b
d0ec409
30f42cd
a129314
d8416cd
d4a84bc
7e9b3a2
78b76a8
88f1bb5
cda56a9
05333ea
4143603
90bec79
cce3f1d
ccb3956
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -144,6 +144,10 @@ private[spark] class BasicExecutorFeatureStep( | |
| .addToLimits("memory", executorMemoryQuantity) | ||
| .addToRequests("cpu", executorCpuQuantity) | ||
| .endResources() | ||
| .addNewEnv() | ||
| .withName(ENV_SPARK_USER) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see that you noted that this is always done across resource managers. What is the reason for that, just wondering? as I introduced it exclusively in the HadoopSteps
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If you don't do this, whatever is the OS user in the container will become the identity used to talk to Hadoop services (when kerberos is not on). In YARN, for example, that would be the "yarn" user. In k8s, with the current image, that would be "root". You probably don't want that by default. We're talking about non-secured Hadoop here, so users can easily override this stuff, but by default let's at least try to identify the user correctly. |
||
| .withValue(Utils.getCurrentUserName()) | ||
| .endEnv() | ||
| .addAllToEnv(executorEnv.asJava) | ||
| .withPorts(requiredPorts.asJava) | ||
| .addToArgs("executor") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,114 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
| package org.apache.spark.deploy.k8s.features | ||
|
|
||
| import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} | ||
| import org.apache.commons.codec.binary.Base64 | ||
| import org.apache.hadoop.security.UserGroupInformation | ||
|
|
||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} | ||
| import org.apache.spark.deploy.k8s.Config._ | ||
| import org.apache.spark.deploy.k8s.Constants._ | ||
| import org.apache.spark.deploy.security.HadoopDelegationTokenManager | ||
| import org.apache.spark.internal.config._ | ||
|
|
||
| /** | ||
| * Delegation token support for Spark apps on kubernetes. | ||
| * | ||
| * When preparing driver resources, this step will generate delegation tokens for the app if | ||
| * they're needed. | ||
| * | ||
| * When preparing pods, this step will mount the delegation token secret (either pre-defined, | ||
| * or generated by this step when preparing the driver). | ||
| */ | ||
| private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDriver: Boolean) | ||
|
||
| extends KubernetesFeatureConfigStep { | ||
|
|
||
| private val existingSecret = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) | ||
| private val existingItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) | ||
| private val shouldCreateTokens = isDriver && !conf.sparkConf.contains(KEYTAB) && | ||
| existingSecret.isEmpty && isSecurityEnabled | ||
|
|
||
| KubernetesUtils.requireBothOrNeitherDefined( | ||
| existingSecret, | ||
| existingItemKey, | ||
| "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + | ||
| " you must also specify the name of the secret", | ||
| "If a secret storing a Kerberos Delegation Token is specified you must also" + | ||
| " specify the item-key where the data is stored") | ||
|
|
||
| private def dtSecretName: String = s"${conf.appResourceNamePrefix}-delegation-tokens" | ||
|
|
||
| override def configurePod(pod: SparkPod): SparkPod = { | ||
| pod.transform { case pod if shouldCreateTokens | existingSecret.isDefined => | ||
| val secretName = existingSecret.getOrElse(dtSecretName) | ||
| val itemKey = existingItemKey.getOrElse(KERBEROS_SECRET_KEY) | ||
|
|
||
| val podWithTokens = new PodBuilder(pod.pod) | ||
| .editOrNewSpec() | ||
| .addNewVolume() | ||
| .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) | ||
| .withNewSecret() | ||
| .withSecretName(secretName) | ||
| .endSecret() | ||
| .endVolume() | ||
| .endSpec() | ||
| .build() | ||
|
|
||
| val containerWithTokens = new ContainerBuilder(pod.container) | ||
| .addNewVolumeMount() | ||
| .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) | ||
| .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) | ||
| .endVolumeMount() | ||
| .addNewEnv() | ||
| .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) | ||
| .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") | ||
| .endEnv() | ||
| .build() | ||
|
|
||
| SparkPod(podWithTokens, containerWithTokens) | ||
| } | ||
| } | ||
|
|
||
| override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { | ||
| if (shouldCreateTokens) { | ||
| val tokenData = createDelegationTokens() | ||
| Seq(new SecretBuilder() | ||
| .withNewMetadata() | ||
| .withName(dtSecretName) | ||
| .endMetadata() | ||
| .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) | ||
| .build()) | ||
| } else { | ||
| Nil | ||
| } | ||
| } | ||
|
|
||
| // Visible for testing. | ||
| def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled() | ||
|
|
||
| // Visible for testing. | ||
| def createDelegationTokens(): Array[Byte] = { | ||
| val tokenManager = new HadoopDelegationTokenManager(conf.sparkConf, | ||
| SparkHadoopUtil.get.newConfiguration(conf.sparkConf)) | ||
| val creds = UserGroupInformation.getCurrentUser().getCredentials() | ||
| tokenManager.obtainDelegationTokens(creds) | ||
| SparkHadoopUtil.get.serialize(creds) | ||
| } | ||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I added this because I started to get tired of code like the following:
To me that's hard to follow and brittle, and this pattern makes things clearer IMO.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would think that this change is out of the scope of this PR, but I do love the use of a PartialFunction here. Thanks for this!