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

Support to run HiveSQLEngine on kerberized YARN #6199

Closed

Conversation

zhouyifan279
Copy link
Contributor

@zhouyifan279 zhouyifan279 commented Mar 19, 2024

🔍 Description

Issue References 🔗

This pull request implement a feature - Run HiveSQLEngine on kerberized YARN

Describe Your Solution 🔧

Introduced two configs:

  • kyuubi.engine.principal
  • kyuubi.engine.keytab

When do submit to a kerberized YARN, submitter uploads kyuubi.engine.keytab to application's staging dir.
YARN NodeManager downloads keytab to AM's working directory. AM logins to Kerberos using the principal and keytab

Note
I've tried to run HiveSQLEngine with only DelegationTokens but failed.

Take SQL SELECT * FROM a as an example:
Hive handles this simple TableScan SQL by reading directly from table's hdfs file.
When Hive invokes FileInputFormat.getSplits during reading, java.io.IOException: Delegation Token can be issued only with kerberos or web authentication will be thrown.
The simplified stacktrace from IDEA is as below:

getDelegationToken:734, DFSClient (org.apache.hadoop.hdfs)
getDelegationToken:2072, DistributedFileSystem (org.apache.hadoop.hdfs)
collectDelegationTokens:108, DelegationTokenIssuer (org.apache.hadoop.security.token)
addDelegationTokens:83, DelegationTokenIssuer (org.apache.hadoop.security.token)
obtainTokensForNamenodesInternal:143, TokenCache (org.apache.hadoop.mapreduce.security)
obtainTokensForNamenodesInternal:102, TokenCache (org.apache.hadoop.mapreduce.security)
obtainTokensForNamenodes:81, TokenCache (org.apache.hadoop.mapreduce.security)
listStatus:221, FileInputFormat (org.apache.hadoop.mapred)
getSplits:332, FileInputFormat (org.apache.hadoop.mapred)
getNextSplits:372, FetchOperator (org.apache.hadoop.hive.ql.exec)
getRecordReader:304, FetchOperator (org.apache.hadoop.hive.ql.exec)
getNextRow:459, FetchOperator (org.apache.hadoop.hive.ql.exec)
pushRow:428, FetchOperator (org.apache.hadoop.hive.ql.exec)
fetch:147, FetchTask (org.apache.hadoop.hive.ql.exec)
getResults:2208, Driver (org.apache.hadoop.hive.ql)
getNextRowSet:494, SQLOperation (org.apache.hive.service.cli.operation)
getNextRowSetInternal:105, HiveOperation (org.apache.kyuubi.engine.hive.operation)

Theoretically, it can be solved by add AM DelegationTokens into
org.apache.hadoop.hive.ql.exec.FetchOperator.job.credentials.
But actually, it is impossible without modifying Hive's source code.

Types of changes 🔖

  • Bugfix (non-breaking change which fixes an issue)
  • New feature (non-breaking change which adds functionality)
  • Breaking change (fix or feature that would cause existing functionality to change)

Test Plan 🧪

Behavior Without This Pull Request ⚰️

HiveSQLEngine can not run on a kerberized YARN

Behavior With This Pull Request 🎉

HiveSQLEngine can run on a kerberized YARN

Related Unit Tests


Checklist 📝

Be nice. Be informative.

@zhouyifan279
Copy link
Contributor Author

cc @pan3793 @yikf

@zhouyifan279 zhouyifan279 force-pushed the kerberized-hive-engine-on-yarn branch from 48241bb to cb06025 Compare March 19, 2024 14:25
@github-actions github-actions bot added the kind:documentation Documentation is a feature! label Mar 19, 2024
@codecov-commenter
Copy link

codecov-commenter commented Mar 19, 2024

Codecov Report

Attention: Patch coverage is 8.69565% with 84 lines in your changes are missing coverage. Please review.

Project coverage is 58.38%. Comparing base (67f099a) to head (9e2d862).
Report is 3 commits behind head on master.

❗ Current head 9e2d862 differs from pull request most recent head 383d1cd. Consider uploading reports for the commit 383d1cd to get more accurate results

Files Patch % Lines
...i/engine/deploy/yarn/EngineYarnModeSubmitter.scala 0.00% 39 Missing ⚠️
.../kyuubi/engine/deploy/yarn/ApplicationMaster.scala 0.00% 22 Missing ⚠️
...apache/kyuubi/engine/hive/HiveProcessBuilder.scala 0.00% 15 Missing ⚠️
...ala/org/apache/kyuubi/util/KyuubiHadoopUtils.scala 0.00% 8 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master    #6199      +/-   ##
============================================
- Coverage     58.58%   58.38%   -0.20%     
  Complexity       24       24              
============================================
  Files           649      649              
  Lines         39379    39468      +89     
  Branches       5415     5425      +10     
============================================
- Hits          23070    23044      -26     
- Misses        13841    13949     +108     
- Partials       2468     2475       +7     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

yarnConf = KyuubiHadoopUtils.newYarnConfiguration(kyuubiConf)
hadoopConf = KyuubiHadoopUtils.newHadoopConf(kyuubiConf)
appUser = kyuubiConf.getOption(KYUUBI_SESSION_USER_KEY).orNull
Copy link
Member

Choose a reason for hiding this comment

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

is this correct for GROUP and SERVER share level?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

KYUUBI_SESSION_USER_KEY is set to EngineRef#appUser when submit Engines. So it's correct for all share levels.

Copy link
Member

Choose a reason for hiding this comment

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

It's weird. I remember we use this key to distinguish the session user and app user in some cases ...

Copy link
Contributor Author

@zhouyifan279 zhouyifan279 Mar 21, 2024

Choose a reason for hiding this comment

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

It is confusing because we currently use KYUUBI_SESSION_USER_KEY in both ProcBuilder and Engine side Operation.
Replacing KYUUBI_SESSION_USER_KEY with another config when used in ProcBuilder can eliminate confusion.
Let's do it in another PR.

Copy link
Member

Choose a reason for hiding this comment

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

SGTM


// Transfer the original user's tokens to the new user, since it may contain needed tokens
// (such as those user to connect to YARN).
// TODO originalCreds may have been expired if Application retries.
Copy link
Member

Choose a reason for hiding this comment

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

IIRC we have disabled the YARN app retry in the Spark engine explicitly.

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 just found we also had disabled retry in EngineYarnModeSubmitter. This comment can be removed.


private def obtainHadoopFsDelegationToken(): Credentials = {
val tokenRenewer = Master.getMasterPrincipal(hadoopConf)
info("Delegation token renewer is: " + tokenRenewer)
Copy link
Member

Choose a reason for hiding this comment

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

Suggested change
info("Delegation token renewer is: " + tokenRenewer)
info("Delegation token renewer is: $tokenRenewer")

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

@pan3793
Copy link
Member

pan3793 commented Mar 20, 2024

Theoretically, it can be solved by add AM DelegationTokens into org.apache.hadoop.mapred.JobConf#credentials.
But actually, it is impossible without modifying Hive's source code.

If possible, would you mind leaving some notes about the invocation chain? It may be asked by future exploreres or contributors

@pan3793 pan3793 requested a review from cxzl25 March 20, 2024 03:43
@cxzl25
Copy link
Contributor

cxzl25 commented Mar 20, 2024

When Hive invokes FileInputFormat#getSplits, java.io.IOException: Delegation Token can be issued only with kerberos or web authentication will be thrown.
Theoretically, it can be solved by add AM DelegationTokens into org.apache.hadoop.mapred.JobConf#credentials.
But actually, it is impossible without modifying Hive's source code.

Agreed, many years ago we removed the TokenCache.obtainTokensForNamenodes logic from Hadoop's FileInputFormat.

// If we passed in a keytab, make sure we copy the keytab to the staging directory on
// HDFS, and setup the relevant environment vars, so the AM can login again.
amKeytabFileName.foreach { kt =>
info("To enable the AM to login from keytab, credentials are being copied over to the AM" +
Copy link
Contributor

Choose a reason for hiding this comment

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

This log seems a bit redundant, because in the Kerberos environment, we explicitly require users to configure this configuration item.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Keytabs are security-sensitive files. I think it's better to tell user why we upload keytabs in place.

newUGI
case _ =>
val appUser = kyuubiConf.getOption(KyuubiReservedKeys.KYUUBI_SESSION_USER_KEY)
require(appUser.isDefined)
Copy link
Contributor

Choose a reason for hiding this comment

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

It will be nice if add error message for the require.

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

val ugi = if (UserGroupInformation.getCurrentUser.getShortUserName == appUser) {
UserGroupInformation.getCurrentUser
} else {
UserGroupInformation.createProxyUser(appUser, UserGroupInformation.getCurrentUser)
Copy link
Contributor

Choose a reason for hiding this comment

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

This also supports proxy users, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes

Copy link
Member

Choose a reason for hiding this comment

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

This should only happen when using TGT cache with YARN mode, though it does not work ... it's good to keep those code path in case some one modifies the Hive source code to make it workable, but we'd better add a warning message to explicitly tell users this is not likely to work.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

More precisely speaking, proxy user mechanism works fine during the submitting progress. AM can be started successfully.
But HiveSQLEngine currently only works with principal & keytab provided.
I think we should check this prerequisite in HiveYarnModeProcessBuilder.

Copy link
Member

Choose a reason for hiding this comment

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

yes, exactly

if (UserGroupInformation.isSecurityEnabled) {
val credentials = obtainHadoopFsDelegationToken()
val serializedCreds = KyuubiHadoopUtils.serializeCredentials(credentials)
amContainer.setTokens(ByteBuffer.wrap(serializedCreds))
Copy link
Contributor

Choose a reason for hiding this comment

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

Why does the ApplicationMaster need to login even if the token is set on this block?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

WithoutUserGroupInformation.loginUserFromKeytab, the login user can only authenticate by tokens.
But HiveSQLEngine requires the login user to authenticate by kerberos tgt to obtain new tokens.

@@ -17,10 +17,12 @@
package org.apache.kyuubi.engine.deploy.yarn
Copy link
Contributor

Choose a reason for hiding this comment

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

The submitter process determine whether to login as well in kerberized environment?

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 "determine whether to login as well" means. Can you elaborate it?

@zhouyifan279 zhouyifan279 force-pushed the kerberized-hive-engine-on-yarn branch from 9d05fa7 to 5d3013a Compare March 21, 2024 12:18
Copy link
Member

@pan3793 pan3793 left a comment

Choose a reason for hiding this comment

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

LGTM, the failed tests should be easy to fix

@pan3793 pan3793 added this to the v1.10.0 milestone Mar 22, 2024
@pan3793
Copy link
Member

pan3793 commented Mar 22, 2024

OK, all Hive tests passed, I'm going to merge this PR.

@zhouyifan279 thanks for your excellent work, and welcome to becoming a Kyuubi committer.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants