-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core #17723
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
Closed
mgummelt
wants to merge
40
commits into
apache:master
from
d2iq-archive:SPARK-20434-refactor-kerberos
Closed
Changes from all commits
Commits
Show all changes
40 commits
Select commit
Hold shift + click to select a range
ce63a9b
[Mesosphere SPARK-126] Move YarnSparkHadoopUtil token helpers into th…
75d849a
[Mesosphere SPARK-126] Add Mesos Kerberos support
35002f2
Par down kerberos support
13981c8
cleanup
af4a3e4
style
5cc66dc
Add MesosSecurityManager
a47c9c0
info logs
c8ec049
style
954eeff
Re-add org.apache.spark.deploy.yarn.security.ServiceCredentialProvide…
2d76928
move YARNHadoopFSCredentialProviderSuite
d8a968d
Move hive test deps to the core module
b8093c8
remove test scope
25d5088
remove test scope
4c387eb
Removed MesosSecurityManager, added RPC call, removed META-INF Servic…
e32afee
add InterfaceStability annotation to ServiceCredentialProvider
be69f5a
Add HadoopAccessManager
55616da
Remove mesos code
240df31
re-add mistakenly removed files
810c6b2
test ConfigurableCredentialManager.obtainUserTokens
ad4e33b
add tests
e15f1ab
rat-excludes
a546aab
fix RAT
d6d21d1
style
092aac7
Remove unneeded import
38adaae
Make ServiceCredentialProvider private
92ac3f0
Addressed style comments
cd58b6c
review comments
bf758e6
style
e820b09
Remove YARNHadoopAccessManagerSuite.scala
7f4ca86
Move thrifts deps back to yarn/pom.xml
cda3538
dependency testing
376dba0
Fix dependency issues, and address style comments
0ffe8f0
Fix scalastyle
7796e14
Add other deps to provided scope
1479c60
Replicate deps in yarn to fix transitivity issue
4d57f7b
update comments
7e2f90d
style
563b80a
Don't throw an exception when Hive classes are not loaded
c684d88
rename
c4149dd
fix docs
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
119 changes: 119 additions & 0 deletions
119
core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.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,119 @@ | ||
| /* | ||
| * 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.security | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.FileSystem | ||
| import org.apache.hadoop.security.Credentials | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.internal.Logging | ||
|
|
||
| /** | ||
| * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to | ||
| * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], | ||
| * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not | ||
| * explicitly disabled. | ||
| * | ||
| * Also, each HadoopDelegationTokenProvider is controlled by | ||
| * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to | ||
| * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be | ||
| * enabled/disabled by the configuration spark.security.credentials.hive.enabled. | ||
| * | ||
| * @param sparkConf Spark configuration | ||
| * @param hadoopConf Hadoop configuration | ||
| * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. | ||
| */ | ||
| private[spark] class HadoopDelegationTokenManager( | ||
| sparkConf: SparkConf, | ||
| hadoopConf: Configuration, | ||
| fileSystems: Set[FileSystem]) | ||
| extends Logging { | ||
|
|
||
| private val deprecatedProviderEnabledConfigs = List( | ||
| "spark.yarn.security.tokens.%s.enabled", | ||
| "spark.yarn.security.credentials.%s.enabled") | ||
| private val providerEnabledConfig = "spark.security.credentials.%s.enabled" | ||
|
|
||
| // Maintain all the registered delegation token providers | ||
| private val delegationTokenProviders = getDelegationTokenProviders | ||
| logDebug(s"Using the following delegation token providers: " + | ||
| s"${delegationTokenProviders.keys.mkString(", ")}.") | ||
|
|
||
| private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { | ||
| val providers = List(new HadoopFSDelegationTokenProvider(fileSystems), | ||
| new HiveDelegationTokenProvider, | ||
| new HBaseDelegationTokenProvider) | ||
|
|
||
| // Filter out providers for which spark.security.credentials.{service}.enabled is false. | ||
| providers | ||
| .filter { p => isServiceEnabled(p.serviceName) } | ||
| .map { p => (p.serviceName, p) } | ||
| .toMap | ||
| } | ||
|
|
||
| def isServiceEnabled(serviceName: String): Boolean = { | ||
| val key = providerEnabledConfig.format(serviceName) | ||
|
|
||
| deprecatedProviderEnabledConfigs.foreach { pattern => | ||
| val deprecatedKey = pattern.format(serviceName) | ||
| if (sparkConf.contains(deprecatedKey)) { | ||
| logWarning(s"${deprecatedKey} is deprecated. Please use ${key} instead.") | ||
| } | ||
| } | ||
|
|
||
| val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => | ||
| sparkConf | ||
| .getOption(pattern.format(serviceName)) | ||
| .map(_.toBoolean) | ||
| .getOrElse(true) | ||
| } | ||
|
|
||
| sparkConf | ||
| .getOption(key) | ||
| .map(_.toBoolean) | ||
| .getOrElse(isEnabledDeprecated) | ||
| } | ||
|
|
||
| /** | ||
| * Get delegation token provider for the specified service. | ||
| */ | ||
| def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { | ||
| delegationTokenProviders.get(service) | ||
| } | ||
|
|
||
| /** | ||
| * Writes delegation tokens to creds. Delegation tokens are fetched from all registered | ||
| * providers. | ||
| * | ||
| * @return Time after which the fetched delegation tokens should be renewed. | ||
| */ | ||
| def obtainDelegationTokens( | ||
| hadoopConf: Configuration, | ||
| creds: Credentials): Long = { | ||
| delegationTokenProviders.values.flatMap { provider => | ||
| if (provider.delegationTokensRequired(hadoopConf)) { | ||
| provider.obtainDelegationTokens(hadoopConf, creds) | ||
| } else { | ||
| logDebug(s"Service ${provider.serviceName} does not require a token." + | ||
| s" Check your configuration to see if security is disabled or not.") | ||
| None | ||
| } | ||
| }.foldLeft(Long.MaxValue)(math.min) | ||
| } | ||
| } |
50 changes: 50 additions & 0 deletions
50
core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.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,50 @@ | ||
| /* | ||
| * 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.security | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.security.Credentials | ||
|
|
||
| /** | ||
| * Hadoop delegation token provider. | ||
| */ | ||
| private[spark] trait HadoopDelegationTokenProvider { | ||
|
|
||
| /** | ||
| * Name of the service to provide delegation tokens. This name should be unique. Spark will | ||
| * internally use this name to differentiate delegation token providers. | ||
| */ | ||
| def serviceName: String | ||
|
|
||
| /** | ||
| * Returns true if delegation tokens are required for this service. By default, it is based on | ||
| * whether Hadoop security is enabled. | ||
| */ | ||
| def delegationTokensRequired(hadoopConf: Configuration): Boolean | ||
|
|
||
| /** | ||
| * Obtain delegation tokens for this service and get the time of the next renewal. | ||
| * @param hadoopConf Configuration of current Hadoop Compatible system. | ||
| * @param creds Credentials to add tokens and security keys to. | ||
| * @return If the returned tokens are renewable and can be renewed, return the time of the next | ||
| * renewal, otherwise None should be returned. | ||
| */ | ||
| def obtainDelegationTokens( | ||
| hadoopConf: Configuration, | ||
| creds: Credentials): Option[Long] | ||
| } | ||
126 changes: 126 additions & 0 deletions
126
core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.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,126 @@ | ||
| /* | ||
| * 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.security | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.util.Try | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.FileSystem | ||
| import org.apache.hadoop.mapred.Master | ||
| import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
| import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier | ||
|
|
||
| import org.apache.spark.SparkException | ||
| import org.apache.spark.internal.Logging | ||
|
|
||
| private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Set[FileSystem]) | ||
| extends HadoopDelegationTokenProvider with Logging { | ||
|
|
||
| // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. | ||
| // If None, no token renewer is specified or no token can be renewed, | ||
| // so we cannot get the token renewal interval. | ||
| private var tokenRenewalInterval: Option[Long] = null | ||
|
|
||
| override val serviceName: String = "hadoopfs" | ||
|
|
||
| override def obtainDelegationTokens( | ||
| hadoopConf: Configuration, | ||
| creds: Credentials): Option[Long] = { | ||
|
|
||
| val newCreds = fetchDelegationTokens( | ||
| getTokenRenewer(hadoopConf), | ||
| fileSystems) | ||
|
|
||
| // Get the token renewal interval if it is not set. It will only be called once. | ||
| if (tokenRenewalInterval == null) { | ||
| tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, fileSystems) | ||
| } | ||
|
|
||
| // Get the time of next renewal. | ||
| val nextRenewalDate = tokenRenewalInterval.flatMap { interval => | ||
| val nextRenewalDates = newCreds.getAllTokens.asScala | ||
| .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) | ||
| .map { token => | ||
| val identifier = token | ||
| .decodeIdentifier() | ||
| .asInstanceOf[AbstractDelegationTokenIdentifier] | ||
| identifier.getIssueDate + interval | ||
| } | ||
| if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) | ||
| } | ||
|
|
||
| creds.addAll(newCreds) | ||
| nextRenewalDate | ||
| } | ||
|
|
||
| def delegationTokensRequired(hadoopConf: Configuration): Boolean = { | ||
| UserGroupInformation.isSecurityEnabled | ||
| } | ||
|
|
||
| private def getTokenRenewer(hadoopConf: Configuration): String = { | ||
| val tokenRenewer = Master.getMasterPrincipal(hadoopConf) | ||
| logDebug("Delegation token renewer is: " + tokenRenewer) | ||
|
|
||
| if (tokenRenewer == null || tokenRenewer.length() == 0) { | ||
| val errorMessage = "Can't get Master Kerberos principal for use as renewer." | ||
| logError(errorMessage) | ||
| throw new SparkException(errorMessage) | ||
| } | ||
|
|
||
| tokenRenewer | ||
| } | ||
|
|
||
| private def fetchDelegationTokens( | ||
| renewer: String, | ||
| filesystems: Set[FileSystem]): Credentials = { | ||
|
|
||
| val creds = new Credentials() | ||
|
|
||
| filesystems.foreach { fs => | ||
| logInfo("getting token for: " + fs) | ||
| fs.addDelegationTokens(renewer, creds) | ||
| } | ||
|
|
||
| creds | ||
| } | ||
|
|
||
| private def getTokenRenewalInterval( | ||
| hadoopConf: Configuration, | ||
| filesystems: Set[FileSystem]): Option[Long] = { | ||
| // We cannot use the tokens generated with renewer yarn. Trying to renew | ||
| // those will fail with an access control issue. So create new tokens with the logged in | ||
| // user as renewer. | ||
| val creds = fetchDelegationTokens( | ||
| UserGroupInformation.getCurrentUser.getUserName, | ||
| filesystems) | ||
|
|
||
| val renewIntervals = creds.getAllTokens.asScala.filter { | ||
| _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] | ||
| }.flatMap { token => | ||
| Try { | ||
| val newExpiration = token.renew(hadoopConf) | ||
| val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] | ||
| val interval = newExpiration - identifier.getIssueDate | ||
| logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") | ||
| interval | ||
| }.toOption | ||
| } | ||
| if (renewIntervals.isEmpty) None else Some(renewIntervals.min) | ||
| } | ||
| } |
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.
Not sure whether it is ok to mark it private? The deprecated one (
trait ServiceCredentialProvider) is not private. cc @vanzin @mridulmUh oh!
There was an error while loading. Please reload this page.
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.
Now that this is private, I've reverted
ServiceCredentialProviderto be non-deprecated.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.
You need to update the PR description since it mentions deprecating the YARN interface still.
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.
fixed