Skip to content
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

Closed

Conversation

mgummelt
Copy link
Contributor

@mgummelt mgummelt commented Apr 21, 2017

What changes were proposed in this pull request?

Move Hadoop delegation token code from spark-yarn to spark-core, so that other schedulers (such as Mesos), may use it. In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private. In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.

Summary:

  • Move registered yarn.security.ServiceCredentialProvider classes from spark-yarn to spark-core. Moved them into a new, private hierarchy under HadoopDelegationTokenProvider. Client code in HadoopDelegationTokenManager now loads credentials from a whitelist of three providers (HadoopFSDelegationTokenProvider, HiveDelegationTokenProvider, HBaseDelegationTokenProvider), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the spark-yarn module.

  • The yarn.security.ServiceCredentialProvider interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations. Client code in YARN now fetches tokens via the new YARNHadoopDelegationTokenManager class, which fetches tokens from the core providers through HadoopDelegationTokenManager, as well as service loads them from yarn.security.ServiceCredentialProvider.

Old Hierarchy:

yarn.security.ServiceCredentialProvider (service loaded)
  HadoopFSCredentialProvider
  HiveCredentialProvider
  HBaseCredentialProvider
yarn.security.ConfigurableCredentialManager

New Hierarchy:

HadoopDelegationTokenManager
HadoopDelegationTokenProvider (not service loaded)
  HadoopFSDelegationTokenProvider
  HiveDelegationTokenProvider
  HBaseDelegationTokenProvider

yarn.security.ServiceCredentialProvider (service loaded)
yarn.security.YARNHadoopDelegationTokenManager

How was this patch tested?

unit tests

@SparkQA
Copy link

SparkQA commented Apr 21, 2017

Test build #76044 has finished for PR 17723 at commit ad4e33b.

  • This patch fails RAT tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 21, 2017

Test build #76045 has finished for PR 17723 at commit e15f1ab.

  • This patch fails RAT tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Apr 21, 2017

Test build #76046 has finished for PR 17723 at commit a546aab.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

core/pom.xml Outdated
-->
<dependency>
<groupId>${hive.group}</groupId>
<artifactId>hive-exec</artifactId>
Copy link
Contributor 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
Contributor 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
Contributor 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.

@SparkQA
Copy link

SparkQA commented Apr 22, 2017

Test build #76047 has finished for PR 17723 at commit d6d21d1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mgummelt
Copy link
Contributor Author

cc @vanzin @jerryshao @skonto

BTW @vanzin, I decided to parameterize HadoopFSCredentialProvider with a new HadoopAccessManager object, for which YARN provides a custom YARNHadoopAccessManager. I did this instead of conditioning on SparkHadoopUtil.get.isYarnMode, since I prefer functional parameterization over global values.

@SparkQA
Copy link

SparkQA commented Jun 9, 2017

Test build #77820 has finished for PR 17723 at commit 7e2f90d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jun 9, 2017

Test build #77822 has finished for PR 17723 at commit 563b80a.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gatorsmile
Copy link
Member

Thanks! Will review it tomorrow.

Copy link
Contributor

@vanzin vanzin left a comment

Choose a reason for hiding this comment

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

Few style nits remaining, otherwise LGTM.

core/pom.xml Outdated
@@ -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
Contributor Author

Choose a reason for hiding this comment

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

fixed

}

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

}

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

creds.addToken(new Text("hive.server2.delegation.token"), hive2Token)
}
} catch {
case NonFatal(e) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Move this catch block to the end of the method?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good catch. fixed.

// public for testing
val credentialProviders = getCredentialProviders

def obtainCredentials(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: fits in one line.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

hadoopConf: Configuration,
creds: Credentials): Long = {

val superInterval = configurableCredentialManager.obtainCredentials(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: fits in one line.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

}

private def loadCredentialProviders: List[ServiceCredentialProvider] = {
ServiceLoader.load(
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: the load(...) call fits in one line.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed. would be nice if scalastyle caught all of these spacing issues.

Copy link
Member

@gatorsmile gatorsmile left a comment

Choose a reason for hiding this comment

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

In the PR description, we need an update. Although this PR does not have a design doc, we still need to clearly describe all the major changes and the relation between these new classes and the existing/new traits.

For example, the relation between ServiceCredentialProvider and HadoopDelegationTokenProvider. Another example, [[HadoopFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] needs to be explained too.

If possible, draw a class hierarchy for them. It will help the future code readers to understand the impact of this PR, after we merge it.

}
}

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

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

private[spark] trait HadoopDelegationTokenProvider {

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

Choose a reason for hiding this comment

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

fixed


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
Contributor Author

Choose a reason for hiding this comment

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

fixed


override def serviceName: String = "hive"

private val classNotFoundErrorStr = "You are attempting to use the HiveCredentialProvider," +
Copy link
Member

Choose a reason for hiding this comment

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

Nit: ," -> , "

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

@@ -368,7 +371,9 @@ private[spark] class Client(
val fs = destDir.getFileSystem(hadoopConf)

// Merge credentials obtained from registered providers
val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials)
val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(
Copy link
Member

Choose a reason for hiding this comment

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

Revert this back?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

// public for testing
val credentialProviders = getCredentialProviders

def obtainCredentials(
Copy link
Member

Choose a reason for hiding this comment

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

YARNConfigurableCredentialManager is not extending ConfigurableCredentialManager . Please add the function description for this function too.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

hadoopConf: Configuration,
fileSystems: Set[FileSystem]) extends Logging {

private val configurableCredentialManager =
Copy link
Member

Choose a reason for hiding this comment

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

Add the comment above this line to explain why YARNConfigurableCredentialManager does not extend ConfigurableCredentialManager

Copy link
Contributor Author

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. Is composition vs. inheritance something that needs to be justified?

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

* 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
Contributor 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.

@mgummelt mgummelt changed the title [SPARK-20434][YARN][CORE] Move kerberos delegation token code from yarn to core [SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core Jun 12, 2017
@mgummelt
Copy link
Contributor Author

@gatorsmile I've expanded the description, and included a before/after class hierarchy.

@SparkQA
Copy link

SparkQA commented Jun 12, 2017

Test build #77941 has finished for PR 17723 at commit c684d88.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mgummelt
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Jun 12, 2017

Test build #77946 has finished for PR 17723 at commit c684d88.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mgummelt
Copy link
Contributor Author

retest this please

@SparkQA
Copy link

SparkQA commented Jun 13, 2017

Test build #77953 has finished for PR 17723 at commit c684d88.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@mgummelt
Copy link
Contributor Author

@gatorsmile All comments addressed. Ready for re-review. Unit tests are failing on master, so I'm waiting for those to be resolved to get a passing build.

@mgummelt
Copy link
Contributor Author

retest this please

@@ -504,10 +504,10 @@ spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://f
```

Spark supports integrating with other security-aware services through Java Services mechanism (see
`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.yarn.security.ServiceCredentialProvider`
`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.security.ServiceCredentialProvider`
Copy link
Member

Choose a reason for hiding this comment

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

revert it back?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

@gatorsmile
Copy link
Member

LGTM except one comment.

cc @vanzin @mridulm @cloud-fan @rxin

@SparkQA
Copy link

SparkQA commented Jun 14, 2017

Test build #78062 has finished for PR 17723 at commit c684d88.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Jun 15, 2017

Test build #78072 has finished for PR 17723 at commit c4149dd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@vanzin
Copy link
Contributor

vanzin commented Jun 15, 2017

LGTM, merging to master.

@asfgit asfgit closed this in a18d637 Jun 15, 2017
@mgummelt
Copy link
Contributor Author

w00t

Thanks everyone.

@mgummelt mgummelt deleted the SPARK-20434-refactor-kerberos branch June 15, 2017 20:48
dataknocker pushed a commit to dataknocker/spark that referenced this pull request Jun 16, 2017
… to core

## What changes were proposed in this pull request?

Move Hadoop delegation token code from `spark-yarn` to `spark-core`, so that other schedulers (such as Mesos), may use it.  In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private.  In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.

Summary:
- Move registered `yarn.security.ServiceCredentialProvider` classes from `spark-yarn` to `spark-core`.  Moved them into a new, private hierarchy under `HadoopDelegationTokenProvider`.  Client code in `HadoopDelegationTokenManager` now loads credentials from a whitelist of three providers (`HadoopFSDelegationTokenProvider`, `HiveDelegationTokenProvider`, `HBaseDelegationTokenProvider`), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the `spark-yarn` module.

- The `yarn.security.ServiceCredentialProvider` interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations.  Client code in YARN now fetches tokens via the new `YARNHadoopDelegationTokenManager` class, which fetches tokens from the core providers through `HadoopDelegationTokenManager`, as well as service loads them from `yarn.security.ServiceCredentialProvider`.

Old Hierarchy:

```
yarn.security.ServiceCredentialProvider (service loaded)
  HadoopFSCredentialProvider
  HiveCredentialProvider
  HBaseCredentialProvider
yarn.security.ConfigurableCredentialManager
```

New Hierarchy:

```
HadoopDelegationTokenManager
HadoopDelegationTokenProvider (not service loaded)
  HadoopFSDelegationTokenProvider
  HiveDelegationTokenProvider
  HBaseDelegationTokenProvider

yarn.security.ServiceCredentialProvider (service loaded)
yarn.security.YARNHadoopDelegationTokenManager
```
## How was this patch tested?

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>
Author: Dr. Stefan Schimanski <sttts@mesosphere.io>

Closes apache#17723 from mgummelt/SPARK-20434-refactor-kerberos.
@ArtRand ArtRand restored the SPARK-20434-refactor-kerberos branch August 21, 2017 20:13
susanxhuynh pushed a commit to mesosphere/spark that referenced this pull request Jan 8, 2018
… to core (Kerberos refactor)

Move Hadoop delegation token code from `spark-yarn` to `spark-core`, so that other schedulers (such as Mesos), may use it.  In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private.  In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.

Summary:
- Move registered `yarn.security.ServiceCredentialProvider` classes from `spark-yarn` to `spark-core`.  Moved them into a new, private hierarchy under `HadoopDelegationTokenProvider`.  Client code in `HadoopDelegationTokenManager` now loads credentials from a whitelist of three providers (`HadoopFSDelegationTokenProvider`, `HiveDelegationTokenProvider`, `HBaseDelegationTokenProvider`), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the `spark-yarn` module.

- The `yarn.security.ServiceCredentialProvider` interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations.  Client code in YARN now fetches tokens via the new `YARNHadoopDelegationTokenManager` class, which fetches tokens from the core providers through `HadoopDelegationTokenManager`, as well as service loads them from `yarn.security.ServiceCredentialProvider`.

Old Hierarchy:

```
yarn.security.ServiceCredentialProvider (service loaded)
  HadoopFSCredentialProvider
  HiveCredentialProvider
  HBaseCredentialProvider
yarn.security.ConfigurableCredentialManager
```

New Hierarchy:

```
HadoopDelegationTokenManager
HadoopDelegationTokenProvider (not service loaded)
  HadoopFSDelegationTokenProvider
  HiveDelegationTokenProvider
  HBaseDelegationTokenProvider

yarn.security.ServiceCredentialProvider (service loaded)
yarn.security.YARNHadoopDelegationTokenManager
```

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>
Author: Dr. Stefan Schimanski <sttts@mesosphere.io>

Closes apache#17723 from mgummelt/SPARK-20434-refactor-kerberos.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants