-
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
Changes from 38 commits
ce63a9b
75d849a
35002f2
13981c8
af4a3e4
5cc66dc
a47c9c0
c8ec049
954eeff
2d76928
d8a968d
b8093c8
25d5088
4c387eb
e32afee
be69f5a
55616da
240df31
810c6b2
ad4e33b
e15f1ab
a546aab
d6d21d1
092aac7
38adaae
92ac3f0
cd58b6c
bf758e6
e820b09
7f4ca86
cda3538
376dba0
0ffe8f0
7796e14
1479c60
4d57f7b
7e2f90d
563b80a
c684d88
c4149dd
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 |
|---|---|---|
|
|
@@ -357,6 +357,34 @@ | |
| <groupId>org.apache.commons</groupId> | ||
| <artifactId>commons-crypto</artifactId> | ||
| </dependency> | ||
|
|
||
| <!-- | ||
| Testing Hive reflection needs hive on the test classpath only, however, while adding hive-exec to the test | ||
| scope works fine in Maven, it causes the sbt build to fail. When the scope is set to "test", SBT seems to fail to | ||
| associate the hive-exec dependency entry here with the dependencyManagement entry in spark-parent, resulting in | ||
| dependency resolution issues. To fix this, hive-exec is instead placed in the "provided" scope. | ||
| --> | ||
| <dependency> | ||
| <groupId>${hive.group}</groupId> | ||
| <artifactId>hive-exec</artifactId> | ||
|
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. I still don't know how to place these in the
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. So I tried a few things, and the one that got me further was just having this: And nix the others. Adding the others in test scope caused some weird error in sbt, even with all dependencies (we have the dependencies you had problems with cached locally). My comment was going to be to add that, then rewrite the code to use the metastore API instead of the All it seems to be doing is making sure the reflection-based code is not completely broken. That is something already, though. So I have two suggestions, in order of preference:
I kinda like the first because it's always good to avoid reflection, and this is a particularly ugly use of it.
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. Thanks for looking into it. Do you know why reflection was used in the first place? Why not just add the Hive dependencies to compile scope? I'm thinking that's what we should do now, and drop reflection. So I'm agreeing with your first bullet point, but proposing that we add the hive deps to
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.
Because technically Hive is an optional dependency for Spark, and moving it to compile scope would break that. (Whether that should change or not is a separate discussion, but probably better not to have it as part of this change.)
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. Alright I added hive-exec to provided scope, and removed the reflection. |
||
| <scope>provided</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>${hive.group}</groupId> | ||
| <artifactId>hive-metastore</artifactId> | ||
| <scope>provided</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.thrift</groupId> | ||
| <artifactId>libthrift</artifactId> | ||
| <scope>provided</scope> | ||
| </dependency> | ||
| <dependency> | ||
| <groupId>org.apache.thrift</groupId> | ||
| <artifactId>libfb303</artifactId> | ||
| <scope>provided</scope> | ||
| </dependency> | ||
|
|
||
| </dependencies> | ||
| <build> | ||
| <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,118 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
| /** | ||
| * A ConfigurableCredentialManager to manage all the registered credential providers and offer | ||
|
||
| * APIs for other modules to obtain credentials as well as renewal time. By default | ||
| * [[HadoopFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will | ||
| * be loaded in if not explicitly disabled. | ||
| * | ||
| * Also each credential provider is controlled by spark.security.credentials.{service}.enabled, | ||
| * it will not be loaded in if set to false. For example, Hive's credential provider | ||
| * [[HiveCredentialProvider]] 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 ConfigurableCredentialManager( | ||
| 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 credential providers | ||
| private val credentialProviders = getCredentialProviders | ||
| logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") | ||
|
|
||
| private def getCredentialProviders: Map[String, HadoopDelegationTokenProvider] = { | ||
| val providers = List(new HadoopFSCredentialProvider(fileSystems), | ||
| new HiveCredentialProvider, | ||
| new HBaseCredentialProvider) | ||
|
|
||
| // Filter out credentials in 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 => | ||
|
Member
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. Nit:
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. Why? It would be a pure, 0-ary function, which is better represented as a val.
Member
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 be used only when |
||
| sparkConf | ||
| .getOption(pattern.format(serviceName)) | ||
| .map(_.toBoolean) | ||
| .getOrElse(true) | ||
| } | ||
|
|
||
| sparkConf | ||
| .getOption(key) | ||
| .map(_.toBoolean) | ||
| .getOrElse(isEnabledDeprecated) | ||
| } | ||
|
|
||
| /** | ||
| * Get credential provider for the specified service. | ||
| */ | ||
| def getServiceCredentialProvider(service: String): Option[HadoopDelegationTokenProvider] = { | ||
|
||
| credentialProviders.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. | ||
|
Member
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 needs to be more accurate to explain the first service provider that needs to renew.
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. I think this is the most accurate and succinct explanation of the contract. Since we aren't returning the renewal time of all tokens, it is true that after the returned timeout, all tokens must be renewed. I could say "Time after which one of the returned tokens must be renewed", but this is a circuitous instruction to the user, since they actually must renew all. |
||
| */ | ||
| def obtainCredentials( | ||
| hadoopConf: Configuration, | ||
| creds: Credentials): Long = { | ||
| credentialProviders.values.flatMap { provider => | ||
| if (provider.credentialsRequired(hadoopConf)) { | ||
| provider.obtainCredentials(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) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -15,7 +15,7 @@ | |
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.deploy.yarn.security | ||
| package org.apache.spark.deploy.security | ||
|
|
||
| import scala.reflect.runtime.universe | ||
| import scala.util.control.NonFatal | ||
|
|
@@ -24,17 +24,16 @@ import org.apache.hadoop.conf.Configuration | |
| import org.apache.hadoop.security.Credentials | ||
| import org.apache.hadoop.security.token.{Token, TokenIdentifier} | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| private[security] class HBaseCredentialProvider extends ServiceCredentialProvider with Logging { | ||
| private[security] class HBaseCredentialProvider | ||
| extends HadoopDelegationTokenProvider with Logging { | ||
|
Member
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. Nit: Shorten it to one line
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. It's too long now with the rename. |
||
|
|
||
| override def serviceName: String = "hbase" | ||
|
|
||
| override def obtainCredentials( | ||
| hadoopConf: Configuration, | ||
| sparkConf: SparkConf, | ||
| creds: Credentials): Option[Long] = { | ||
| try { | ||
| val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) | ||
|
|
||
| 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 { | ||
|
Member
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.
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. Now that this is private, I've reverted
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. You need to update the PR description since it mentions deprecating the YARN interface still.
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. fixed |
||
|
|
||
| /** | ||
| * Name of the service to provide credentials. This name should be unique, Spark internally will | ||
|
||
| * use this name to differentiate credential provider. | ||
| */ | ||
| def serviceName: String | ||
|
|
||
| /** | ||
| * Returns true if credentials are required for this service. By default, it is based on whether | ||
| * Hadoop security is enabled. | ||
|
||
| */ | ||
| def credentialsRequired(hadoopConf: Configuration): Boolean | ||
|
|
||
| /** | ||
| * Obtain credentials 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 this Credential is renewable and can be renewed, return the time of the next | ||
| * renewal, otherwise None should be returned. | ||
| */ | ||
| def obtainCredentials( | ||
| hadoopConf: Configuration, | ||
| creds: Credentials): Option[Long] | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,124 @@ | ||
| /* | ||
| * 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 | ||
|
|
||
|
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. nit: too many blank lines
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. Fixed (I was in PEP8 mode) |
||
| private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) | ||
| extends HadoopDelegationTokenProvider with Logging { | ||
| // Token renewal interval, this value will be set in the first call, | ||
|
||
| // if None means no token renewer specified or no token can be renewed, | ||
| // so cannot get token renewal interval. | ||
| private var tokenRenewalInterval: Option[Long] = null | ||
|
|
||
| override val serviceName: String = "hadoopfs" | ||
|
|
||
| override def obtainCredentials( | ||
| 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 credentialsRequired(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) | ||
| } | ||
| } | ||
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.
Minor, but this comment is a little stale now after all the changes. There's no reflection anymore, so the "provided" scope is just so we can still package Spark without Hive.
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