Skip to content
Closed
Show file tree
Hide file tree
Changes from 38 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…
Feb 10, 2016
75d849a
[Mesosphere SPARK-126] Add Mesos Kerberos support
Feb 10, 2016
35002f2
Par down kerberos support
Apr 17, 2017
13981c8
cleanup
Apr 17, 2017
af4a3e4
style
Apr 17, 2017
5cc66dc
Add MesosSecurityManager
Apr 18, 2017
a47c9c0
info logs
Apr 18, 2017
c8ec049
style
Apr 18, 2017
954eeff
Re-add org.apache.spark.deploy.yarn.security.ServiceCredentialProvide…
Apr 18, 2017
2d76928
move YARNHadoopFSCredentialProviderSuite
Apr 18, 2017
d8a968d
Move hive test deps to the core module
Apr 19, 2017
b8093c8
remove test scope
Apr 19, 2017
25d5088
remove test scope
Apr 19, 2017
4c387eb
Removed MesosSecurityManager, added RPC call, removed META-INF Servic…
Apr 20, 2017
e32afee
add InterfaceStability annotation to ServiceCredentialProvider
Apr 20, 2017
be69f5a
Add HadoopAccessManager
Apr 21, 2017
55616da
Remove mesos code
Apr 21, 2017
240df31
re-add mistakenly removed files
Apr 21, 2017
810c6b2
test ConfigurableCredentialManager.obtainUserTokens
Apr 21, 2017
ad4e33b
add tests
Apr 21, 2017
e15f1ab
rat-excludes
Apr 21, 2017
a546aab
fix RAT
Apr 21, 2017
d6d21d1
style
Apr 21, 2017
092aac7
Remove unneeded import
Apr 24, 2017
38adaae
Make ServiceCredentialProvider private
May 18, 2017
92ac3f0
Addressed style comments
May 18, 2017
cd58b6c
review comments
May 22, 2017
bf758e6
style
May 23, 2017
e820b09
Remove YARNHadoopAccessManagerSuite.scala
May 23, 2017
7f4ca86
Move thrifts deps back to yarn/pom.xml
May 31, 2017
cda3538
dependency testing
Jun 2, 2017
376dba0
Fix dependency issues, and address style comments
Jun 2, 2017
0ffe8f0
Fix scalastyle
Jun 2, 2017
7796e14
Add other deps to provided scope
Jun 2, 2017
1479c60
Replicate deps in yarn to fix transitivity issue
Jun 5, 2017
4d57f7b
update comments
Jun 6, 2017
7e2f90d
style
Jun 8, 2017
563b80a
Don't throw an exception when Hive classes are not loaded
Jun 8, 2017
c684d88
rename
Jun 12, 2017
c4149dd
fix docs
Jun 15, 2017
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 28 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -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
Copy link
Contributor

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.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

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>
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I still don't know how to place these in the test scope, which is where they belong. See my comment here: https://github.com/apache/spark/pull/17665/files#r112337820

Copy link
Contributor

Choose a reason for hiding this comment

The 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:

    <dependency>
      <groupId>${hive.group}</groupId>
      <artifactId>hive-metastore</artifactId>
      <scope>test</scope>
    </dependency>

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 Hive class from hive-exec... but then I noticed that test is not doing much, because there are no metastore servers to talk to. It's even there, hardcoded in the test:

  test("obtain tokens For HiveMetastore") {
    ...
    credentials.getAllTokens.size() should be (0)

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:

  • add the dependencies in "provided" scope, and change the code to use actual types and not reflection. Because the classes may not exist at runtime, that means having to handle NoClassDefFoundError in creative ways.

  • keep the reflection code, and remove this test. Or maybe move it to a separate module as others have suggested.

I kinda like the first because it's always good to avoid reflection, and this is a particularly ugly use of it.

Copy link
Author

Choose a reason for hiding this comment

The 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 compile rather than provided.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not just add the Hive dependencies to compile scope?

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.)

Copy link
Author

@mgummelt mgummelt Jun 2, 2017

Choose a reason for hiding this comment

The 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>
Expand Down
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
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Author

Choose a reason for hiding this comment

The 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.
*
* 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 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't look correct. What I'd expect:

  • try the new key, if it is set, then don't print out anything.
  • if it's not set, then try the deprecated keys (in some order) and use the first that is set, printing a warning message

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).

Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Author

@mgummelt mgummelt May 18, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in case the new setting is not configured, the message is misleading (since you are not using the new setting).

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can't you merge this with the other code below? Something like:

deprecatedProviderEnabledConfigs
  .map {
    // get value and print deprecated message
  }
  .somethingThatReturnsTrueOrFalse

Copy link
Author

Choose a reason for hiding this comment

The 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. Please use ${key} instead.")
}
}

val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: def

Copy link
Author

Choose a reason for hiding this comment

The 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.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will be used only when key is not defined in SparkConf.

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] = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In this ConfigurableCredentialManager , we are using the terminology ServiceCredentialProvider. However, ServiceCredentialProvider is a Yarn-specific trait. It is confusing when reading the codes.

If possible, we need to change the names and terms used in the class ConfigurableCredentialManager

Copy link
Author

@mgummelt mgummelt Jun 12, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You're right. I made a few changes to naming:

*CredentialProvider -> *DelegationTokenProvider
ConfigurableCredentialManager -> HadoopDelegationTokenManager
YARNConfigurableCredentialManager -> YARNHadoopDelegationTokenManager

and updated a bunch of comments.

cc @vanzin. you might be interested in this since these renames are non-trivial

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.
Copy link
Member

Choose a reason for hiding this comment

The 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.

Copy link
Author

Choose a reason for hiding this comment

The 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
Expand Up @@ -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
Expand All @@ -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 {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Shorten it to one line

Copy link
Author

Choose a reason for hiding this comment

The 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)
Expand Down
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 {
Copy link
Member

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 @mridulm

Copy link
Author

@mgummelt mgummelt May 22, 2017

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 ServiceCredentialProvider to be non-deprecated.

Copy link
Contributor

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.

Copy link
Author

Choose a reason for hiding this comment

The 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
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: unique, -> unique.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

* 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.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It sounds like you update the comments in the original file, but you did not update the comment here.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comments are identical. Can you be more specific?

*/
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

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: too many blank lines

Copy link
Author

Choose a reason for hiding this comment

The 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,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How about?

The token renewal interval will be set in the first call.

Copy link
Author

Choose a reason for hiding this comment

The 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,
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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent extra level

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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent extra level

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)
}
}
Loading