-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core #17723
Changes from 23 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. | ||
It doesn't need the spark hive modules, so the -Phive flag is not checked. | ||
--> | ||
<dependency> | ||
<groupId>${hive.group}</groupId> | ||
<artifactId>hive-exec</artifactId> | ||
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 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. 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 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.) 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. |
||
</dependency> | ||
<dependency> | ||
<groupId>${hive.group}</groupId> | ||
<artifactId>hive-metastore</artifactId> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.thrift</groupId> | ||
<artifactId>libthrift</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.thrift</groupId> | ||
<artifactId>libfb303</artifactId> | ||
<scope>test</scope> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.hadoop</groupId> | ||
<artifactId>hadoop-yarn-api</artifactId> | ||
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. The reason why I had introduced I would expect 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. Core depends on The distinction was useful when Spark supported Hadoop 1.x (no YARN), but right now, it's mostly theoretical - every Spark build has YARN classes, regardless of whether the YARN module is included. 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 pointing it out, I just noticed that hadoop-client deps have changed in the long interim since I last looked at it :-) |
||
</dependency> | ||
</dependencies> | ||
<build> | ||
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,3 @@ | ||
org.apache.spark.deploy.security.HadoopFSCredentialProvider | ||
org.apache.spark.deploy.security.HBaseCredentialProvider | ||
org.apache.spark.deploy.security.HiveCredentialProvider |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* 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 java.util.ServiceLoader | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.security.{Credentials, UserGroupInformation} | ||
|
||
import org.apache.spark.SparkConf | ||
import org.apache.spark.deploy.SparkHadoopUtil | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.util.Utils | ||
|
||
/** | ||
* A ConfigurableCredentialManager to manage all the registered credential providers and offer | ||
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 comment is now a little incorrect, because this is not extensible anymore. 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 |
||
* 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, any plugged-in credential provider wants to be | ||
* managed by ConfigurableCredentialManager needs to implement [[ServiceCredentialProvider]] | ||
* interface and put into resources/META-INF/services to be loaded by ServiceLoader. | ||
* | ||
* 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. | ||
*/ | ||
private[spark] class ConfigurableCredentialManager( | ||
sparkConf: SparkConf, | ||
hadoopConf: Configuration, | ||
hadoopAccessManager: HadoopAccessManager) | ||
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(", ")}.") | ||
|
||
def this(sparkConf: SparkConf, hadoopConf: Configuration) { | ||
this(sparkConf, hadoopConf, new DefaultHadoopAccessManager(hadoopConf)) | ||
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. Could we remove this constructor 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'd be open to removing this one, but see below for why we should keep the other one. |
||
} | ||
|
||
def this(sparkConf: SparkConf) { | ||
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. Also remove this one? 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. Mesos will make use of this simpler constructor: https://github.com/apache/spark/pull/17665/files#diff-387c5d0c916278495fc28420571adf9eR59 |
||
this(sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) | ||
} | ||
|
||
private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { | ||
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. either remove the 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. removed |
||
val providers = loadCredentialProviders | ||
|
||
// Filter out credentials in which spark.security.credentials.{service}.enabled is false. | ||
providers | ||
.filter(p => isServiceEnabled(p.serviceName)) | ||
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: style is I see this in other places so please go through all your changes and fix all instances. 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: style is 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 all occurrences I could find. |
||
.map(p => (p.serviceName, p)) | ||
.toMap | ||
} | ||
|
||
protected def isServiceEnabled(serviceName: String): Boolean = { | ||
val key = providerEnabledConfig.format(serviceName) | ||
|
||
deprecatedProviderEnabledConfigs.foreach { pattern => | ||
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 doesn't look correct. What I'd expect:
This block of code is printing a warning whenever the old settings exist, regardless of the new setting being configured; and in case the new setting is not configured, the message is misleading (since you are not using the new setting). 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. To add to @vanzin's comment, if both new and old config's are present - it would be good to warn user. 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.
Yea, this is actually how it worked previously, too, so I was erring on the side of retaining the old, incorrect behavior. I'll change the message. 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. Can't you merge this with the other code below? Something like:
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 prefer to keep functional an non-functional code separate when possible. I can change it if you feel otherwise. |
||
val deprecatedKey = pattern.format(serviceName) | ||
if (sparkConf.contains(deprecatedKey)) { | ||
logWarning(s"${deprecatedKey} is deprecated, using ${key} instead") | ||
} | ||
} | ||
|
||
val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => | ||
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: 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. 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) | ||
} | ||
|
||
private def loadCredentialProviders: List[ServiceCredentialProvider] = { | ||
ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) | ||
.asScala.toList | ||
} | ||
|
||
/** | ||
* Get credential provider for the specified service. | ||
*/ | ||
def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = { | ||
credentialProviders.get(service) | ||
} | ||
|
||
/** | ||
* Writes delegation tokens to creds. Delegation tokens are fetched from all registered | ||
* providers. | ||
* | ||
* @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable, | ||
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. Sentence is a little redundant. 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. Agreed. Fixed. |
||
* otherwise the nearest renewal time of any credentials will be returned. | ||
*/ | ||
def obtainCredentials( | ||
hadoopConf: Configuration, | ||
creds: Credentials): Long = { | ||
credentialProviders.values.flatMap { provider => | ||
if (provider.credentialsRequired(hadoopConf)) { | ||
provider.obtainCredentials(hadoopConf, hadoopAccessManager, 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 |
---|---|---|
@@ -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.security | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.FileSystem | ||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.security.UserGroupInformation | ||
|
||
class DefaultHadoopAccessManager(hadoopConf: Configuration) extends HadoopAccessManager { | ||
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. Needs 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 |
||
|
||
def getTokenRenewer: String = { | ||
UserGroupInformation.getCurrentUser.getShortUserName | ||
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 general, do we want to depend on UGI for spark security ? Can we externalize use of UGI from core ? Yes, it has leaked into core in a few places - which is unfortunate IMO. 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 resolved by our previous discussions. |
||
} | ||
|
||
def hadoopFSsToAccess: Set[Path] = { | ||
Set(FileSystem.get(hadoopConf).getHomeDirectory) | ||
} | ||
|
||
def getTokenRenewalInterval: Option[Long] = { | ||
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. Please add 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. done |
||
None | ||
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. Please write a comment to explain why we return None. |
||
} | ||
} |
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.security | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
|
||
/** | ||
* Methods in [[HadoopAccessManager]] return scheduler-specific information related to how Hadoop | ||
* delegation tokens should be fetched. | ||
*/ | ||
private[spark] trait HadoopAccessManager { | ||
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. Do we have any design doc about this new trait? cc @kanzhang who is the original creator of delegation token concepts in Hadoop. 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. No design doc. 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. For such big PRs, we normally prefer to reviewing the design doc at first. When the PR does not have a design doc, it could slow down the review. 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. I'll make sure to do that in the future. We happened to already have most of this code written internally before upstreaming to Apache, which is why I submitted before going through a design review. As for this trait, I introduced it in order to parameterize I had originally tried to introduce both a 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 explaining it. Just need to understand the design of this trait. Why we pick these three functions here? 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 just removed this entire trait. The purpose of the trait was to parameterize |
||
|
||
/** The user allowed to renew delegation tokens */ | ||
def getTokenRenewer: String | ||
|
||
/** The renewal interval, or [[None]] if the token shouldn't be renewed */ | ||
def getTokenRenewalInterval: Option[Long] | ||
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. A basic question. Conceptually, I am wondering why 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. Is Mesos behaving differently from Yarn when getting token renewal intervals? 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. That's a good point. They don't. I have it factored out like this because the code which computes it depends on I'll go ahead and remove this function from this trait and put it back in |
||
|
||
/** The set of hadoop file systems to fetch delegation tokens for */ | ||
def hadoopFSsToAccess: Set[Path] | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
/* | ||
* 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 org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.security.Credentials | ||
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier | ||
|
||
import org.apache.spark.internal.Logging | ||
|
||
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 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 | ||
extends ServiceCredentialProvider with Logging { | ||
// Token renewal interval, this value will be set in the first call, | ||
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. How about?
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 |
||
// 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, | ||
hadoopAccessManager: HadoopAccessManager, | ||
creds: Credentials): Option[Long] = { | ||
|
||
// NameNode to access, used to get tokens from different FileSystems | ||
val tmpCreds = new Credentials() | ||
val tokenRenewer = hadoopAccessManager.getTokenRenewer | ||
hadoopAccessManager.hadoopFSsToAccess.foreach { dst => | ||
val dstFs = dst.getFileSystem(hadoopConf) | ||
logInfo("getting token for: " + dst) | ||
dstFs.addDelegationTokens(tokenRenewer, tmpCreds) | ||
} | ||
|
||
// Get the token renewal interval if it is not set. It will only be called once. | ||
if (tokenRenewalInterval == null) { | ||
tokenRenewalInterval = hadoopAccessManager.getTokenRenewalInterval | ||
} | ||
|
||
// Get the time of next renewal. | ||
val nextRenewalDate = tokenRenewalInterval.flatMap { interval => | ||
val nextRenewalDates = tmpCreds.getAllTokens.asScala | ||
.filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) | ||
.map { t => | ||
val identifier = t.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] | ||
identifier.getIssueDate + interval | ||
} | ||
if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) | ||
} | ||
|
||
creds.addAll(tmpCreds) | ||
nextRenewalDate | ||
} | ||
} |
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.
Is that possible to avoid adding these dependencies to core? Does it make sense to do the test in
org.apache.spark.sql.hive
? cc @vanzin @mridulmThere 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.
The code is in core, so the test should be in core.
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 we workaround it? It's weird to test code of sql module in hive module, but it's weirder to let sql module depends on hive module. If we can't work around it, I'd like to move the test to hive module.
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.
I'm not sure what you mean. This is the core module, not the sql module.
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.
oh sorry I made a mistake, but it's much weirder to let core module depend on hive module...
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 went back in forth on this in the main thread, which is admittedly long and hard to follow at this point. Here's the same concern laid out by mridulm: #17723 (comment)
We ultimately decided that it would be acceptable to add these dependencies, so long as we don't expose them in the public Spark interfaces. Here's a comment to that effect: #17723 (comment)
The reason they're required is that this PR moves the Hadoop delegation token providers from the yarn module into the core module, so that other resource managers, such as Mesos, can also fetch delegation tokens to access secure hadoop services. One of those delegation token providers is Hive: https://github.com/apache/spark/pull/17723/files#diff-8be1059872a069d1bb4c5fc3ca394968
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.
Also, to clarify, this is not adding a dependency to the Spark hive module. It's adding a dependency to Apache Hive.