This repository was archived by the owner on Jan 9, 2020. It is now read-only.
forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 117
Secure HDFS Support #414
Merged
erikerlandson
merged 43 commits into
apache-spark-on-k8s:hdfs-kerberos-support
from
bloomberg:secure-hdfs-support4
Sep 28, 2017
Merged
Secure HDFS Support #414
Changes from all commits
Commits
Show all changes
43 commits
Select commit
Hold shift + click to select a range
ea9e516
Initial architecture design for HDFS support
ifilonenko 47ea307
Minor styling
ifilonenko 60a19ca
Added proper logic for mounting ConfigMaps
ifilonenko 1d0175a
styling
ifilonenko 163193a
modified otherKubernetesResource logic
ifilonenko 8381fa6
fixed Integration tests and modified HADOOP_CONF_DIR variable to be F…
ifilonenko d4b1a68
setting HADOOP_CONF_DIR env variables
ifilonenko 0bba092
Included integration tests for Stage 1
ifilonenko 06df962
Initial Kerberos support
ifilonenko d7f54dd
initial Stage 2 architecture using deprecated 2.1 methods
ifilonenko d3c5a03
Added current, BROKEN, integration test environment for review
ifilonenko d7441ba
working hadoop cluster
ifilonenko 04eed68
Using locks and monitors to ensure proper configs for setting up kerb…
ifilonenko 62354eb
working Stage 2
ifilonenko 514ac19
documentation
ifilonenko 3fbf88c
Integration Stages 1,2 and 3
ifilonenko b321436
further testing work
ifilonenko b6912d2
fixing imports
ifilonenko c6b11f8
Stage 3 Integration tests pass
ifilonenko 1e71ca7
uncommented SparkDockerBuilder
ifilonenko 350c8ed
testing fix
ifilonenko 5e4051c
handled comments and increased test hardening
ifilonenko 8338fdb
Solve failing integration test problem and lower TIMEOUT time
ifilonenko d6d0945
modify security.authoization
ifilonenko e3f14e1
Modifying HADOOP_CONF flags
ifilonenko 61a7414
Refactored tests and included modifications to pass all tests regardl…
ifilonenko 7a0b4e4
Adding unit test and one more integration test
ifilonenko 8dacb19
completed unit tests w/o UGI mocking
ifilonenko d9b7b50
cleanup and various small fixes
ifilonenko d53a50f
added back sparkdockerbuilder images
ifilonenko 499b037
merge issues
ifilonenko ffe7891
address initial comments and scalastyle issues
ifilonenko 6efa379
addresses comments from PR
ifilonenko 6052a13
mocking hadoopUGI
ifilonenko f9ca47d
Fix executor env to include simple authn
kimoonkim 91e364c
Merge remote-tracking branch 'bloomberg/secure-hdfs-support4' into pr…
kimoonkim 4fe86f0
Merge pull request #1 from kimoonkim/pr-414
ifilonenko d2c8649
Fix a bug in executor env handling
kimoonkim 4780878
Merge remote-tracking branch 'bloomberg/secure-hdfs-support4' into pr…
kimoonkim 17f2702
Merge pull request #2 from kimoonkim/pr-414
ifilonenko b566fa9
Fix a bug in how the driver sets simple authn
kimoonkim 726ff64
Merge pull request #3 from kimoonkim/pr-414
ifilonenko 2d48613
handling Pr comments
ifilonenko File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
86 changes: 86 additions & 0 deletions
86
...bernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopConfBootstrap.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) | ||
} | ||
} |
78 changes: 78 additions & 0 deletions
78
...ers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/HadoopUGIUtil.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
/* | ||
* 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.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} | ||
|
||
import scala.util.Try | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.FileSystem | ||
import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
import org.apache.hadoop.security.token.{Token, TokenIdentifier} | ||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier | ||
|
||
|
||
// Function of this class is merely for mocking reasons | ||
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) | ||
|
||
def dfsAddDelegationToken(hadoopConf: Configuration, renewer: String, creds: Credentials) | ||
: Iterable[Token[_ <: TokenIdentifier]] = | ||
FileSystem.get(hadoopConf).addDelegationTokens(renewer, creds) | ||
|
||
def getCurrentTime: Long = System.currentTimeMillis() | ||
|
||
// Functions that should be in Core with Rebase to 2.3 | ||
@deprecated("Moved to core in 2.2", "2.2") | ||
def getTokenRenewalInterval( | ||
renewedTokens: Iterable[Token[_ <: TokenIdentifier]], | ||
hadoopConf: Configuration): Option[Long] = { | ||
val renewIntervals = renewedTokens.filter { | ||
_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]} | ||
.flatMap { token => | ||
Try { | ||
val newExpiration = token.renew(hadoopConf) | ||
val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] | ||
val interval = newExpiration - identifier.getIssueDate | ||
interval | ||
}.toOption} | ||
if (renewIntervals.isEmpty) None else Some(renewIntervals.min) | ||
} | ||
|
||
@deprecated("Moved to core in 2.2", "2.2") | ||
def serialize(creds: Credentials): Array[Byte] = { | ||
val byteStream = new ByteArrayOutputStream | ||
val dataStream = new DataOutputStream(byteStream) | ||
creds.writeTokenStorageToStream(dataStream) | ||
byteStream.toByteArray | ||
} | ||
|
||
@deprecated("Moved to core in 2.2", "2.2") | ||
def deserialize(tokenBytes: Array[Byte]): Credentials = { | ||
val creds = new Credentials() | ||
creds.readTokenStorageStream(new DataInputStream(new ByteArrayInputStream(tokenBytes))) | ||
creds | ||
} | ||
} |
76 changes: 76 additions & 0 deletions
76
...s/core/src/main/scala/org/apache/spark/deploy/kubernetes/KerberosTokenConfBootstrap.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 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 KerberosTokenConfBootstrap 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) | ||
} | ||
} |
28 changes: 28 additions & 0 deletions
28
...ernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithMainContainer.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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) |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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?