Skip to content

Conversation

@jerryshao
Copy link
Contributor

@jerryshao jerryshao commented Jul 6, 2016

What changes were proposed in this pull request?

Add a configurable token manager for Spark on running on yarn.

Current Problems

  1. Supported token provider is hard-coded, currently only hdfs, hbase and hive are supported and it is impossible for user to add new token provider without code changes.
  2. Also this problem exits in timely token renewer and updater.

Changes In This Proposal

In this proposal, to address the problems mentioned above and make the current code more cleaner and easier to understand, mainly has 3 changes:

  1. Abstract a ServiceTokenProvider as well as ServiceTokenRenewable interface for token provider. Each service wants to communicate with Spark through token way needs to implement this interface.
  2. Provide a ConfigurableTokenManager to manage all the register token providers, also token renewer and updater. Also this class offers the API for other modules to obtain tokens, get renewal interval and so on.
  3. Implement 3 built-in token providers HDFSTokenProvider, HiveTokenProvider and HBaseTokenProvider to keep the same semantics as supported today. Whether to load in these built-in token providers is controlled by configuration "spark.yarn.security.tokens.${service}.enabled", by default for all the built-in token providers are loaded.

Behavior Changes

For the end user there's no behavior change, we still use the same configuration spark.yarn.security.tokens.${service}.enabled to decide which token provider is enabled (hbase or hive).

For user implemented token provider (assume the name of token provider is "test") needs to add into this class should have two configurations:

  1. spark.yarn.security.tokens.test.enabled to true
  2. spark.yarn.security.tokens.test.class to the full qualified class name.

So we still keep the same semantics as current code while add one new configuration.

Current Status

  • token provider interface and management framework.
  • implement built-in token providers (hdfs, hbase, hive).
  • Coverage of unit test.
  • Integrated test with security cluster.

How was this patch tested?

Unit test and integrated test.

Please suggest and review, any comment is greatly appreciated.

@SparkQA
Copy link

SparkQA commented Jul 6, 2016

Test build #61830 has finished for PR 14065 at commit 90f194e.

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

@jerryshao jerryshao changed the title [SPARK-16342][YARN][WIP] Add a configurable token manager for Spark running on YARN [SPARK-14743][YARN][WIP] Add a configurable token manager for Spark running on YARN Jul 7, 2016
@SparkQA
Copy link

SparkQA commented Jul 7, 2016

Test build #61896 has finished for PR 14065 at commit 2b17f2e.

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

hadoopUtil.obtainTokenForHiveMetastore(sparkConf, freshHadoopConf, tempCreds)
hadoopUtil.obtainTokenForHBase(sparkConf, freshHadoopConf, tempCreds)
hdfsTokenProvider(sparkConf).setNameNodesToAccess(sparkConf, Set(dst))
hdfsTokenProvider(sparkConf).setTokenRenewer(null)
Copy link
Contributor

@tgravescs tgravescs Jul 7, 2016

Choose a reason for hiding this comment

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

why are we setting this to null here?

@tgravescs
Copy link
Contributor

I took a quick look through.

It might be nice to think about how we could handle other credentials.

For instance Apache Kafka currently doesn't have tokens so you need keytab or TGT and jaas conf file. Yes they are adding tokens but in in the mean time how does that work. Are there other services similar to that.

Can we handle things other then Tokens? it does appear that I could implement my own ServiceTokenProvider that goes off to really any service and I can put things into the Credentials object as Token or as a Secret so perhaps we are covered here. But perhaps that means we should rename things to be obtainCredentials rather then obtainTokens.

Are there specific services you were thinking about here? We could atleast use those as examples to make sure interface fits those.

@SparkQA
Copy link

SparkQA commented Jul 7, 2016

Test build #61912 has finished for PR 14065 at commit c9d9ed0.

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

private[spark] override def startExecutorDelegationTokenRenewer(sparkConf: SparkConf): Unit = {
tokenRenewer = Some(new ExecutorDelegationTokenUpdater(sparkConf, conf))
tokenRenewer.get.updateCredentialsIfRequired()
configurableTokenManager(sparkConf).delegationTokenUpdater(conf)
Copy link
Contributor

Choose a reason for hiding this comment

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

I find this syntax a little confusing. You're calling configurableTokenManager(sparkConf) in a bunch of different places. To me that looks like either:

  • each call is creating a new token manager
  • there's some cache of token managers somewhere keyed by the spark configuration passed here

Neither sounds good to me. And the actual implementation is actually neither: there's a single token manager singleton that is instantiated in the first call to configurableTokenManager.

Why doesn't Client instantiate a token manager in its constructor instead? Another option is to have an explicit method in ConfigurableTokenManager to initialize the singleton, although I'm not a fan of singletons in general.

@jerryshao
Copy link
Contributor Author

Thanks a lot @tgravescs and @vanzin for your suggestions, I will change the codes accordingly, greatly appreciate your comments.

@SparkQA
Copy link

SparkQA commented Jul 12, 2016

Test build #62148 has finished for PR 14065 at commit 0fbf25b.

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

@jerryshao
Copy link
Contributor Author

@tgravescs and @vanzin , these days I did some code refactoring work on this patch. Here listed changes I did compared to previous code:

  1. Change the interface ServiceTokenProvider to ServiceCredentialProvider with the main method changed to obtainCredentials as suggested in comments.

    Now since we're not limiting to tokens, so instead of obtaining tokens, here changed to obtain credentials. And the method obtainCredentials is defined as:

     def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Option[Long]

    Here the return value Option[Long] means the time of next renewal, return Some(Long) if this credential is renewable, otherwise returns None.

    Also remove several redundant methods like get token renewal interval and so on.

  2. Change ConfigurableTokenManager to ConfigurableCredentialManager to manager all the credential providers.

  3. Change the way to load credential providers to ServiceLoader as suggested in comments.

  4. Change initialization way from singleton to normal way.

  5. Change the mechanism of checking credentials in AMDelegationRenewer and ExecutorDelegationTokenUpdate. Since now we can get the time of next renewal, so we use this to decide when to wake up to check the new credentials.

Please help to review, thanks a lot for your time and greatly appreciate your comments.

@jerryshao
Copy link
Contributor Author

Also thinking about one example to land this feature, I think Kafka might be one candidate, they also have delegation token based proposal KIP-48.

@SparkQA
Copy link

SparkQA commented Jul 12, 2016

Test build #62149 has finished for PR 14065 at commit 60a275f.

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

@SparkQA
Copy link

SparkQA commented Jul 13, 2016

Test build #62219 has finished for PR 14065 at commit a8141a5.

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

@jerryshao jerryshao changed the title [SPARK-14743][YARN][WIP] Add a configurable token manager for Spark running on YARN [SPARK-14743][YARN] Add a configurable token manager for Spark running on YARN Jul 15, 2016
Copy link
Contributor

Choose a reason for hiding this comment

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

Instead of adding these files like this, isn't it possible to just say META-INF/services/*?

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 followed the convention of other services like DataSource which also needs to be excluded from rat check.

@SparkQA
Copy link

SparkQA commented Aug 9, 2016

Test build #63405 has finished for PR 14065 at commit 71df428.

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

credential provider.

```
spark.yarn.access.namenodes hdfs://ireland.example.org:8020/,hdfs://frankfurt.example.org:8020/
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, it seems like this example should be before the paragraph you're adding.

@vanzin
Copy link
Contributor

vanzin commented Aug 9, 2016

Looks fine. There are some possible enhancements (e.g. what looks like some code repetition in the HDFS provider, neither Hive nor HBase return a token renewal time, etc) but those can be done separately.

@tgravescs did you have any remaining comments?

@tgravescs
Copy link
Contributor

all my original comments were addressed and I won't have time to do another review until next week so I'm good with it if you are.

@SparkQA
Copy link

SparkQA commented Aug 10, 2016

Test build #63516 has finished for PR 14065 at commit bce8cd6.

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

@vanzin
Copy link
Contributor

vanzin commented Aug 10, 2016

Merging to master.

@asfgit asfgit closed this in ab648c0 Aug 10, 2016
asfgit pushed a commit that referenced this pull request Feb 21, 2017
…e time

## What changes were proposed in this pull request?

In #14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` .

This PR fixes this mistake.

## How was this patch tested?

existing test

cc jerryshao vanzin

Author: Kent Yao <[email protected]>

Closes #16955 from yaooqinn/cred_update.

(cherry picked from commit 7363dde)
Signed-off-by: Marcelo Vanzin <[email protected]>
asfgit pushed a commit that referenced this pull request Feb 21, 2017
…e time

## What changes were proposed in this pull request?

In #14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` .

This PR fixes this mistake.

## How was this patch tested?

existing test

cc jerryshao vanzin

Author: Kent Yao <[email protected]>

Closes #16955 from yaooqinn/cred_update.
Yunni pushed a commit to Yunni/spark that referenced this pull request Feb 27, 2017
…e time

## What changes were proposed in this pull request?

In apache#14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` .

This PR fixes this mistake.

## How was this patch tested?

existing test

cc jerryshao vanzin

Author: Kent Yao <[email protected]>

Closes apache#16955 from yaooqinn/cred_update.
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.

4 participants