-
Notifications
You must be signed in to change notification settings - Fork 117
[WIP] Use HDFS Delegation Token in driver/executor pods as part of Secure HDFS Support #379
Changes from all commits
48533ff
3c3331a
f2a4033
aa04b04
0141c0a
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 |
---|---|---|
|
@@ -43,6 +43,13 @@ package object constants { | |
s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" | ||
private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" | ||
|
||
// Hadoop credentials secrets for the Spark app. | ||
private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" | ||
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 put these into /etc/hadoop/ so that only root access will be allowed to this folder. thoughts and reasoning for location? Also, should this be customizable in 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. In YARN, the token file is placed into a work dir for the executors. So I don't think /etc/hadoop is a good location. And root access or not is irrelevant in pods since everything runs as root. I chose this dir because we have the other code using it. I think it's better to follow the convention here. From line 30 above:
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.
|
||
private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME = "hadoop-token-file" | ||
private[spark] val SPARK_APP_HADOOP_TOKEN_FILE_PATH = | ||
s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$SPARK_APP_HADOOP_TOKEN_FILE_SECRET_NAME" | ||
private[spark] val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" | ||
|
||
// Default and fixed ports | ||
private[spark] val SUBMISSION_SERVER_PORT = 7077 | ||
private[spark] val DEFAULT_DRIVER_PORT = 7078 | ||
|
@@ -69,6 +76,7 @@ package object constants { | |
private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" | ||
private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" | ||
private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" | ||
private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" | ||
|
||
// Bootstrapping dependencies with the init-container | ||
private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,7 +20,7 @@ import org.apache.spark.SparkConf | |
import org.apache.spark.deploy.kubernetes.ConfigurationUtils | ||
import org.apache.spark.deploy.kubernetes.config._ | ||
import org.apache.spark.deploy.kubernetes.constants._ | ||
import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} | ||
import org.apache.spark.deploy.kubernetes.submit.submitsteps._ | ||
import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator | ||
import org.apache.spark.launcher.SparkLauncher | ||
import org.apache.spark.util.Utils | ||
|
@@ -94,6 +94,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( | |
submissionSparkConf) | ||
val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( | ||
submissionSparkConf, kubernetesResourceNamePrefix) | ||
val hadoopCredentialsStep = new DriverHadoopCredentialsStep(submissionSparkConf) | ||
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. This will need to be slightly refactored after merging this in after PR 373 because of the hadoopStepsOrchestrator that is leveraged. 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. Good to know. Thanks. |
||
val pythonStep = mainAppResource match { | ||
case PythonMainAppResource(mainPyResource) => | ||
Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) | ||
|
@@ -131,6 +132,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( | |
Seq( | ||
initialSubmissionStep, | ||
kubernetesCredentialsStep, | ||
hadoopCredentialsStep, | ||
dependencyResolutionStep) ++ | ||
initContainerBootstrapStep.toSeq ++ | ||
pythonStep.toSeq | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* 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.kubernetes.submit | ||
|
||
import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} | ||
|
||
import org.apache.spark.deploy.kubernetes.constants._ | ||
|
||
object HadoopSecretUtil { | ||
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. any reason for why this needs to be a seperate util and cant just be as part of a step. where is this being re-used? 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. on another note the pod and container will be shared by the PodWithDetachedMainContainer() so it will need to be refactored when PRs are merged 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. This is used by both the driver and executors. 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. See #373 for how one can share a module between the driver and the executors. But we also can think about how to do this better since the abstraction is a little difficult to reason about right now. We do a similar operation with setting the init-container bootstrap on the executors. |
||
|
||
def configurePod(secretNameOption: Option[String], pod: Pod) : Pod = { | ||
secretNameOption.map { secret => | ||
new PodBuilder(pod) | ||
.editOrNewSpec() | ||
.addNewVolume() | ||
.withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) | ||
.withNewSecret() | ||
.withSecretName(secret) | ||
.endSecret() | ||
.endVolume() | ||
.endSpec() | ||
.build() | ||
}.getOrElse(pod) | ||
} | ||
|
||
def configureContainer(secretNameOption: Option[String], | ||
containerSpec: Container) : Container = { | ||
secretNameOption.map { secret => | ||
new ContainerBuilder(containerSpec) | ||
.addNewVolumeMount() | ||
.withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) | ||
.withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) | ||
.endVolumeMount() | ||
.addNewEnv() | ||
.withName(ENV_HADOOP_TOKEN_FILE_LOCATION) | ||
.withValue(SPARK_APP_HADOOP_TOKEN_FILE_PATH) | ||
.endEnv() | ||
.build() | ||
}.getOrElse(containerSpec) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,38 @@ | ||
/* | ||
* 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.kubernetes.submit.submitsteps | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.deploy.kubernetes.config._ | ||
import org.apache.spark.deploy.kubernetes.submit.HadoopSecretUtil | ||
|
||
private[spark] class DriverHadoopCredentialsStep(submissionSparkConf: SparkConf) | ||
extends DriverConfigurationStep { | ||
|
||
private val maybeMountedHadoopSecret = submissionSparkConf.get(MOUNTED_HADOOP_SECRET_CONF) | ||
|
||
override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { | ||
val podWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, | ||
driverSpec.driverPod) | ||
val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( | ||
maybeMountedHadoopSecret, | ||
driverSpec.driverContainer) | ||
driverSpec.copy( | ||
driverPod = podWithMountedHadoopToken, | ||
driverContainer = containerWithMountedHadoopToken) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -37,7 +37,7 @@ import org.apache.spark.{SparkContext, SparkEnv, SparkException} | |
import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} | ||
import org.apache.spark.deploy.kubernetes.config._ | ||
import org.apache.spark.deploy.kubernetes.constants._ | ||
import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil | ||
import org.apache.spark.deploy.kubernetes.submit.{HadoopSecretUtil, InitContainerUtil} | ||
import org.apache.spark.network.netty.SparkTransportConf | ||
import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient | ||
import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} | ||
|
@@ -130,6 +130,8 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( | ||
ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) | ||
|
||
private val maybeMountedHadoopSecret = conf.get(MOUNTED_HADOOP_SECRET_CONF) | ||
|
||
private val driverPod = try { | ||
kubernetesClient.pods().inNamespace(kubernetesNamespace). | ||
withName(kubernetesDriverPodName).get() | ||
|
@@ -582,9 +584,14 @@ private[spark] class KubernetesClusterSchedulerBackend( | |
|
||
val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( | ||
executorPodWithInitContainer, nodeToLocalTaskCount) | ||
val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) | ||
val executorPodWithMountedHadoopToken = HadoopSecretUtil.configurePod(maybeMountedHadoopSecret, | ||
executorPodWithNodeAffinity) | ||
val containerWithMountedHadoopToken = HadoopSecretUtil.configureContainer( | ||
maybeMountedHadoopSecret, initBootstrappedExecutorContainer) | ||
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. Same as above with detachedContainer logic 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 would recommend to leverage a Bootstrap method similair to how SparkPodInitBootStrap functions that could be passed in as an Option 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. Yes. This builder code would deserve some refactoring. But I'd like to put it off until the prototype works end-to-end and we see clearly what is needed. |
||
|
||
val resolvedExecutorPod = new PodBuilder(executorPodWithMountedHadoopToken) | ||
.editSpec() | ||
.addToContainers(initBootstrappedExecutorContainer) | ||
.addToContainers(containerWithMountedHadoopToken) | ||
.endSpec() | ||
.build() | ||
try { | ||
|
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.
We could make the key configurable as well so that it doesn't always have to be "hadoop-token-file".