-
Notifications
You must be signed in to change notification settings - Fork 118
Secure HDFS Support #414
Secure HDFS Support #414
Changes from 42 commits
ea9e516
47ea307
60a19ca
1d0175a
163193a
8381fa6
d4b1a68
0bba092
06df962
d7f54dd
d3c5a03
d7441ba
04eed68
62354eb
514ac19
3fbf88c
b321436
b6912d2
c6b11f8
1e71ca7
350c8ed
5e4051c
8338fdb
d6d0945
e3f14e1
61a7414
7a0b4e4
8dacb19
d9b7b50
d53a50f
499b037
ffe7891
6efa379
6052a13
f9ca47d
91e364c
4fe86f0
d2c8649
4780878
17f2702
b566fa9
726ff64
2d48613
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 |
---|---|---|
|
@@ -783,6 +783,50 @@ from the other deployment modes. See the [configuration page](configuration.html | |
</td> | ||
</tr> | ||
<tr> | ||
<td><code>spark.kubernetes.kerberos.enabled</code></td> | ||
<td>false</td> | ||
<td> | ||
Specify whether your job requires a Kerberos Authentication to access HDFS. By default, we | ||
will assume that you will not require secure HDFS access. | ||
</td> | ||
</tr> | ||
<tr> | ||
<td><code>spark.kubernetes.kerberos.keytab</code></td> | ||
<td>(none)</td> | ||
<td> | ||
Assuming you have set <code>spark.kubernetes.kerberos.enabled</code> to be true. This will let you specify | ||
the location of your Kerberos keytab to be used in order to access Secure HDFS. This is optional as you | ||
may login by running <code>kinit</code> before running the spark-submit, and the submission client | ||
will look within your local TGT cache to resolve this. | ||
</td> | ||
</tr> | ||
<tr> | ||
<td><code>spark.kubernetes.kerberos.principal</code></td> | ||
<td>(none)</td> | ||
<td> | ||
Assuming you have set <code>spark.kubernetes.kerberos.enabled</code> to be true. This will let you specify | ||
your Kerberos principal that you wish to use to access Secure HDFS. This is optional as you | ||
may login by running <code>kinit</code> before running the spark-submit, and the submission client | ||
will look within your local TGT cache to resolve this. | ||
</td> | ||
</tr> | ||
<tr> | ||
<td><code>spark.kubernetes.kerberos.tokensecret.name</code></td> | ||
<td>(none)</td> | ||
<td> | ||
Assuming you have set <code>spark.kubernetes.kerberos.enabled</code> to be true. This will let you specify | ||
the name of the secret where your existing delegation token data is stored. You must also specify the | ||
item key <code>spark.kubernetes.kerberos.tokensecret.itemkey</code> where your data is stored on the secret. | ||
</td> | ||
</tr> | ||
<tr> | ||
<td><code>spark.kubernetes.kerberos.tokensecret.itemkey</code></td> | ||
<td>spark.kubernetes.kerberos.dt.label</td> | ||
<td> | ||
Assuming you have set <code>spark.kubernetes.kerberos.enabled</code> to be true. This will let you specify | ||
the label within the pre-specified secret where the data of your existing delegation token data is stored. | ||
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. s/label/data item key name/ |
||
We have a default value of <code>spark.kubernetes.kerberos.tokensecret.itemkey</code> should you not include it. But | ||
you should always include this if you are proposing a pre-existing secret contain the delegation token data. | ||
<td><code>spark.executorEnv.[EnvironmentVariableName]</code></td> | ||
<td>(none)</td> | ||
<td> | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -100,6 +100,12 @@ | |
<dependency> | ||
<groupId>com.fasterxml.jackson.jaxrs</groupId> | ||
<artifactId>jackson-jaxrs-json-provider</artifactId> | ||
<exclusions> | ||
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. Hmm. Do we still need this? |
||
<exclusion> | ||
<groupId>javax.ws.rs</groupId> | ||
<artifactId>jsr311-api</artifactId> | ||
</exclusion> | ||
</exclusions> | ||
</dependency> | ||
<dependency> | ||
<groupId>javax.ws.rs</groupId> | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,86 @@ | ||
/* | ||
* 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 | ||
|
||
import java.io.File | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import io.fabric8.kubernetes.api.model.{ContainerBuilder, KeyToPathBuilder, PodBuilder} | ||
|
||
import org.apache.spark.deploy.kubernetes.constants._ | ||
import org.apache.spark.internal.Logging | ||
|
||
/** | ||
* This is separated out from the HadoopConf steps API because this component can be reused to | ||
* set up the Hadoop Configuration for executors as well. | ||
*/ | ||
private[spark] trait HadoopConfBootstrap { | ||
/** | ||
* Bootstraps a main container with the ConfigMaps containing Hadoop config files | ||
* mounted as volumes and an ENV variable pointing to the mounted file. | ||
*/ | ||
def bootstrapMainContainerAndVolumes( | ||
originalPodWithMainContainer: PodWithMainContainer) | ||
: PodWithMainContainer | ||
} | ||
|
||
private[spark] class HadoopConfBootstrapImpl( | ||
hadoopConfConfigMapName: String, | ||
hadoopConfigFiles: Seq[File], | ||
hadoopUGI: HadoopUGIUtil) extends HadoopConfBootstrap with Logging{ | ||
|
||
override def bootstrapMainContainerAndVolumes( | ||
originalPodWithMainContainer: PodWithMainContainer) | ||
: PodWithMainContainer = { | ||
logInfo("HADOOP_CONF_DIR defined. Mounting HDFS specific .xml files") | ||
val keyPaths = hadoopConfigFiles.map(file => | ||
new KeyToPathBuilder() | ||
.withKey(file.toPath.getFileName.toString) | ||
.withPath(file.toPath.getFileName.toString) | ||
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. Key and path are exactly same. Is this intended? If yes, why not use a set? 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. It is intended to make my life easier when looking at the configmap and seeing the data be: core-site.xml |
||
.build()).toList | ||
val hadoopSupportedPod = new PodBuilder(originalPodWithMainContainer.pod) | ||
.editSpec() | ||
.addNewVolume() | ||
.withName(HADOOP_FILE_VOLUME) | ||
.withNewConfigMap() | ||
.withName(hadoopConfConfigMapName) | ||
.withItems(keyPaths.asJava) | ||
.endConfigMap() | ||
.endVolume() | ||
.endSpec() | ||
.build() | ||
val mainContainerWithMountedHadoopConf = new ContainerBuilder( | ||
originalPodWithMainContainer.mainContainer) | ||
.addNewVolumeMount() | ||
.withName(HADOOP_FILE_VOLUME) | ||
.withMountPath(HADOOP_CONF_DIR_PATH) | ||
.endVolumeMount() | ||
.addNewEnv() | ||
.withName(ENV_HADOOP_CONF_DIR) | ||
.withValue(HADOOP_CONF_DIR_PATH) | ||
.endEnv() | ||
.addNewEnv() | ||
.withName(ENV_SPARK_USER) | ||
.withValue(hadoopUGI.getShortName) | ||
.endEnv() | ||
.build() | ||
originalPodWithMainContainer.copy( | ||
pod = hadoopSupportedPod, | ||
mainContainer = mainContainerWithMountedHadoopConf) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,27 @@ | ||
/* | ||
* 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 | ||
|
||
import org.apache.hadoop.security.UserGroupInformation | ||
|
||
private[spark] class HadoopUGIUtil{ | ||
def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser | ||
def getShortName: String = getCurrentUser.getShortUserName | ||
def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled | ||
def loginUserFromKeytabAndReturnUGI(principal: String, keytab: String): UserGroupInformation = | ||
UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,76 @@ | ||
/* | ||
* 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 | ||
|
||
import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} | ||
|
||
import org.apache.spark.deploy.kubernetes.constants._ | ||
import org.apache.spark.internal.Logging | ||
|
||
|
||
/** | ||
* This is separated out from the HadoopConf steps API because this component can be reused to | ||
* mounted the DT secret for executors as well. | ||
*/ | ||
private[spark] trait KerberosTokenBootstrapConf { | ||
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. Make the trail name match the file name, KerberosTokenConfBootstrap? |
||
// Bootstraps a main container with the Secret mounted as volumes and an ENV variable | ||
// pointing to the mounted file containing the DT for Secure HDFS interaction | ||
def bootstrapMainContainerAndVolumes( | ||
originalPodWithMainContainer: PodWithMainContainer) | ||
: PodWithMainContainer | ||
} | ||
|
||
private[spark] class KerberosTokenConfBootstrapImpl( | ||
secretName: String, | ||
secretItemKey: String, | ||
userName: String) extends KerberosTokenBootstrapConf with Logging{ | ||
|
||
|
||
override def bootstrapMainContainerAndVolumes( | ||
originalPodWithMainContainer: PodWithMainContainer) | ||
: PodWithMainContainer = { | ||
logInfo("Mounting HDFS DT from Secret for Secure HDFS") | ||
val dtMountedPod = new PodBuilder(originalPodWithMainContainer.pod) | ||
.editOrNewSpec() | ||
.addNewVolume() | ||
.withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) | ||
.withNewSecret() | ||
.withSecretName(secretName) | ||
.endSecret() | ||
.endVolume() | ||
.endSpec() | ||
.build() | ||
val mainContainerWithMountedKerberos = new ContainerBuilder( | ||
originalPodWithMainContainer.mainContainer) | ||
.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/$secretItemKey") | ||
.endEnv() | ||
.addNewEnv() | ||
.withName(ENV_SPARK_USER) | ||
.withValue(userName) | ||
.endEnv() | ||
.build() | ||
originalPodWithMainContainer.copy( | ||
pod = dtMountedPod, | ||
mainContainer = mainContainerWithMountedKerberos) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
/* | ||
* 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 | ||
|
||
import io.fabric8.kubernetes.api.model.{Container, Pod} | ||
|
||
/** | ||
* The purpose of this case class is so that we can package together | ||
* the driver pod with its container so we can bootstrap and modify | ||
* the class instead of each component seperately | ||
*/ | ||
private[spark] case class PodWithMainContainer( | ||
pod: Pod, | ||
mainContainer: Container) |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -45,9 +45,6 @@ package object constants { | |
|
||
// Hadoop credentials secrets for the Spark app. | ||
private[spark] val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" | ||
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 | ||
|
@@ -79,6 +76,7 @@ package object constants { | |
private[spark] val ENV_JAVA_OPT_PREFIX = "SPARK_JAVA_OPT_" | ||
private[spark] val ENV_MOUNTED_FILES_FROM_SECRET_DIR = "SPARK_MOUNTED_FILES_FROM_SECRET_DIR" | ||
private[spark] val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" | ||
private[spark] val ENV_SPARK_USER = "SPARK_USER" | ||
|
||
// Bootstrapping dependencies with the init-container | ||
private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" | ||
|
@@ -101,6 +99,26 @@ package object constants { | |
private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" | ||
private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" | ||
|
||
// Hadoop Configuration | ||
private[spark] val HADOOP_FILE_VOLUME = "hadoop-properties" | ||
private[spark] val HADOOP_CONF_DIR_PATH = "/etc/hadoop/conf" | ||
private[spark] val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" | ||
private[spark] val HADOOP_CONF_DIR_LOC = "spark.kubernetes.hadoop.conf.dir" | ||
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. Are these 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 an interesting point. This will break out system, true. But this is our current point of communication towards the executors |
||
private[spark] val HADOOP_CONFIG_MAP_SPARK_CONF_NAME = | ||
"spark.kubernetes.hadoop.executor.hadoopconfigmapname" | ||
|
||
// Kerberos Configuration | ||
private[spark] val HADOOP_KERBEROS_SECRET_NAME = | ||
"spark.kubernetes.kerberos.dt" | ||
private[spark] val HADOOP_KERBEROS_CONF_SECRET = | ||
"spark.kubernetes.kerberos.secretname" | ||
private[spark] val HADOOP_KERBEROS_CONF_ITEM_KEY = | ||
"spark.kubernetes.kerberos.itemkeyname" | ||
private[spark] val KERBEROS_SECRET_LABEL_PREFIX = | ||
"hadoop-tokens" | ||
private[spark] val SPARK_HADOOP_PREFIX = "spark.hadoop." | ||
private[spark] val HADOOP_SECURITY_AUTHENTICATION = | ||
SPARK_HADOOP_PREFIX + "hadoop.security.authentication" | ||
// Bootstrapping dependencies via a secret | ||
private[spark] val MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH = "/etc/spark-submitted-files" | ||
|
||
|
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.
Can you mention that this is optional in case you want to use pre-existing secret and a new secret will be automatically created otherwise?