-
Notifications
You must be signed in to change notification settings - Fork 961
[KYUUBI #7156] Support update HBase delegation token for Spark SQL Engine #7182
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
base: master
Are you sure you want to change the base?
Conversation
…xy spark applications which need to access secure hbase cluster
kyuubi-server/pom.xml
Outdated
<dependency> | ||
<groupId>org.apache.hbase</groupId> | ||
<artifactId>hbase-client</artifactId> | ||
<version>2.5.12-hadoop3</version> |
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 remember HBase has shaded-client, is it sufficient for requesting DT? I am afraid pulling Hive/HBase deps into the server classpath is the beginning of a nightmare ...
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.
It seems ok to use this shaded hbase client.
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-shaded-client</artifactId>
<version>2.5.12-hadoop3</version>
</dependency>
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.
define version at root pom.xml
's dependencyManagement
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.
BTW, why is version 2.5.12 chosen?
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 use 2.5.12 and 1.4.9 in our production environment. And 2.5.12 client seems able to fetch token from servers of those two versions.
...main/resources/META-INF/services/org.apache.kyuubi.credentials.HadoopDelegationTokenProvider
Outdated
Show resolved
Hide resolved
doAsProxyUser(owner) { | ||
try { | ||
info(s"Getting hbase token for ${owner} ...") | ||
val conn = ConnectionFactory.createConnection(hbaseConf) |
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.
If we use reflection to obtain tokens similar to Spark, this will avoid direct introduction of dependencies.
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.
Introducing HBase's dependencies directly is based on the following considerations :
- Using reflection may reduce code readability and make unit testing more difficult to implement.
- Furthermore, it increases the deployment complexity of the Kyuubi Server, as users need to place related HBase jars into the classpath of the Kyuubi Server.
- The shaded hbase client package seems to have clean depdency.
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.
@Z1Wu I think we'd better align with Spark, we can add a maven property hbase.scope
, default value is provided
, then the official release does not ship HBase jars, and users who use HBase can build with -Dhbase.scope=compile
.
If this approach is adopted, we must ensure it won't throw linkage issues without HBase jars under the Kyuubi server's classpath
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.
Sounds reasonable, I use property name hbase.deps.scope
to align with spark.
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.
Have you tested running Kyuubi server without HBase deps? Will it cause linkage error on initializing HBaseDelegationTokenProvider
?
Codecov Report❌ Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #7182 +/- ##
======================================
Coverage 0.00% 0.00%
======================================
Files 695 696 +1
Lines 43433 43500 +67
Branches 5887 5884 -3
======================================
- Misses 43433 43500 +67 ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
All failed unit tests have been fixed. Please review this patch at your convenience. Thank you. |
kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProvider.scala
Outdated
Show resolved
Hide resolved
final val HBASE_KERBEROS_REALM = "TEST.ORG" | ||
final val HBASE_KERBEROS_PRINCIPAL = "hbase/localhost" | ||
final val HBASE_KERBEROS_KEYTAB = "/opt/hbase/conf/hbase.keytab" | ||
final val DOCKER_IMAGE_NAME = "z1wu97/kyuubi-hbase-cluster:latest" |
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.
Does this docker have a github repository
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.
This must be either built in the CI workflow, or managed by Kyuubi PMC. I can help create an image after you freeze the Dockerfile
override def initialize(hadoopConf: Configuration, kyuubiConf: KyuubiConf): Unit = { | ||
this.kyuubiConf = kyuubiConf | ||
this.hbaseConf = hadoopConf | ||
this.tokenRequired = hbaseConf.get("hbase.security.authentication") == "kerberos" |
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 the config value case sensitive? IIRC, hadoop.security.authentication
is case insensitive
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.
Yes, you are right. This config hbase.security.authentication
is case insensitive.
The failed unit tests seem irrelevant with this patch. Please review again at your convenience. Thank you. |
# --build-arg HBASE_VERSION="2.5.12" \ | ||
# --build-arg ZK_VERSION="3.8.4" \ | ||
# --file build/Dockerfile.HBase \ | ||
# --tag nekyuubi/kyuubi-hbase-cluster:<tag> \ |
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 will upload the image after you fix the style
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.
@pan3793 All style related comment have been resolved, PTAL. And the failed unit tests seem irrelevant with the modification.
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.
nekyuubi/kyuubi-hbase-cluster:latest
is available on dockerhub
kyuubi-server/src/test/scala/org/apache/kyuubi/WithSecuredHBaseContainer.scala
Outdated
Show resolved
Hide resolved
…eContainer.scala Co-authored-by: Cheng Pan <[email protected]>
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.
LGTM, @cxzl25 @zhouyifan279 do you want to take another look?
|
||
override def initialize(hadoopConf: Configuration, kyuubiConf: KyuubiConf): Unit = { | ||
this.kyuubiConf = kyuubiConf | ||
this.hbaseConf = hadoopConf |
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.
hadoopConf
does not contain HBase properties. HBase properties is in hbase-site.xml and needs to be loaded using class HBaseConfiguration
.
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.
Currently I enable this feature by adding hbase related configurations in kyuubi-default.conf
. And hadoop conf is initialized as below:
// org.apache.kyuubi.credentials.HadoopCredentialsManager#initialize
override def initialize(conf: KyuubiConf): Unit = {
hadoopConf = KyuubiHadoopUtils.newHadoopConf(conf)
providers = HadoopCredentialsManager.loadProviders(conf)
.filter { case (_, provider) =>
provider.initialize(hadoopConf, conf)
val required = provider.delegationTokensRequired()
if (!required) {
warn(s"Service ${provider.serviceName} does not require a token." +
s" Check your configuration to see if security is disabled or not." +
s" If security is enabled, some configurations of ${provider.serviceName} " +
s" might be missing, please check the configurations in " +
s" https://kyuubi.readthedocs.io/en/master/security" +
s"/hadoop_credentials_manager.html#required-security-configs")
provider.close()
}
required
}
}
doAsProxyUser(owner) { | ||
try { | ||
info(s"Getting HBase delegation token for ${owner} ...") | ||
val conn = ConnectionFactory.createConnection(hbaseConf) |
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.
Connection should be closed after obtaining token.
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.
Nice catch!
I have re-read Spark's implementation of calling Hbase, and there is a close connection.
org.apache.hadoop.hbase.security.token.TokenUtil#obtainToken(org.apache.hadoop.conf.Configuration)
try (Connection connection = ConnectionFactory.createConnection(conf)) {
return obtainToken(connection);
}
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.
Nice catch, fixed by referring to Spark's implementation.
Why are the changes needed?
How was this patch tested?
unit test
Add a unit test which fetching delegation token from a hbase cluster created by in docker containter.
manual integrate test with hbase deps
compile with
-Dhbase.deps.scope=compile
run kyuubi server with hbase related conf
request hbase token
run sample sql
manual integrate test without hbase deps
compile without
-Dhbase.deps.scope=compile
only fetch HMS token and hdfs token
run sample query
Was this patch authored or co-authored using generative AI tooling?
No