This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new d3f716e215 [KYUUBI #7156] Support update HBase delegation token for 
Spark SQL Engine
d3f716e215 is described below

commit d3f716e215d3e2a6607c5ff888e0e5d4f764c148
Author: wuziyi <[email protected]>
AuthorDate: Wed Sep 17 10:13:40 2025 +0800

    [KYUUBI #7156] Support update HBase delegation token for Spark SQL Engine
    
    ### Why are the changes needed?
    
    - There are some SQLs need to access hbase such as SQL read from Hive-Hbase 
Storage Handler tables. And for proxy-user basd SparkSQLEngine, it is unable to 
renew delegation token by itself. It would be convenient if kyuubi can renew 
hbase delegation token.
    - Currently kyuubi server only supports updating hdfs and hms delegation 
token.
    
    ### 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`
    
    ```
    ./build/dist --tgz --spark-provided --flink-provided --hive-provided --name 
spark3.5 -Pspark3.5 -Dhbase.deps.scope=compile
    ```
    
    <img width="489" height="35" alt="image" 
src="https://github.com/user-attachments/assets/c47303b9-1c6e-44a8-801f-de45ed732be1";
 />
    
    run kyuubi server with hbase related conf
    
    ``` conf
    hbase.zookeeper.quorum xxxx
    hbase.security.authentication kerberos
    hbase.master.kerberos.principal xxx
    hbase.regionserver.kerberos.principal xxx
    ```
    
    request hbase token
    
    <img width="1632" height="278" alt="image" 
src="https://github.com/user-attachments/assets/e73b9534-5c3e-47f4-b1b4-df4ffef900a5";
 />
    
    run sample sql
    
    <img width="1898" height="723" alt="image" 
src="https://github.com/user-attachments/assets/b7aa53a4-dd13-47fc-9218-31e01f43056e";
 />
    
    #### manual integrate test  without hbase deps
    
    compile without `-Dhbase.deps.scope=compile`
    
    ```
    ./build/dist --tgz --spark-provided --flink-provided --hive-provided --name 
spark3.5 -Pspark3.5
    ```
    
    <img width="496" height="48" alt="image" 
src="https://github.com/user-attachments/assets/3d07b0ad-fa81-4aa9-aa98-436908afe721";
 />
    
    only fetch HMS token and hdfs token
    
    <img width="1468" height="58" alt="image" 
src="https://github.com/user-attachments/assets/4665f6f8-99df-4671-8d44-b64ffc8ac98d";
 />
    
    run sample query
    
    <img width="1345" height="571" alt="image" 
src="https://github.com/user-attachments/assets/c33e6d34-d3d6-4429-9612-347fcee832c3";
 />
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No
    
    Closes #7182 from Z1Wu/feat/hbase_delegation_token_support.
    
    Closes #7156
    
    3c61b4f65 [wuziyi] [fix] format
    e58288ed0 [wuziyi] [fix] close connection
    9c8e7bfd1 [z1wu] Update 
kyuubi-server/src/test/scala/org/apache/kyuubi/WithSecuredHBaseContainer.scala
    10d7dbb71 [wuziyi] [fix] log format
    95c7700df [wuziyi] fix format
    aa32571b9 [wuziyi] fix NPE
    781b50294 [wuziyi] format
    426d3004c [wuziyi] exclude unnecessary dependency
    92cb10256 [wuziyi] update
    a58dff752 [wuziyi] update
    aad4c2eba [wuziyi] update
    9244dac19 [wuziyi] [fix] spi order
    e15f855c9 [wuziyi] fix
    8ba55cb9e [wuziyi] switch to hbase-shaded-client-byo-hadoop client jar (byo 
stands for `bring your own`)
    6a65ecfaa [wuziyi] update (revert to unshaded hbase client for test)
    a86eb2a44 [wuziyi] update
    1769c5e4c [wuziyi] update
    737326081 [wuziyi] [feat] add hbase delegation token manager to support 
long running proxy spark applications which need to access secure hbase cluster
    
    Lead-authored-by: wuziyi <[email protected]>
    Co-authored-by: wuziyi <[email protected]>
    Co-authored-by: z1wu <[email protected]>
    Signed-off-by: Cheng Pan <[email protected]>
---
 build/Dockerfile.HBase                             | 319 +++++++++++++++++++++
 kyuubi-server/pom.xml                              |   5 +
 ...yuubi.credentials.HadoopDelegationTokenProvider |   1 +
 .../credentials/HBaseDelegationTokenProvider.scala |  69 +++++
 .../apache/kyuubi/WithSecuredHBaseContainer.scala  | 150 ++++++++++
 .../HBaseDelegationTokenProviderSuite.scala        |  55 ++++
 pom.xml                                            |  15 +
 7 files changed, 614 insertions(+)

diff --git a/build/Dockerfile.HBase b/build/Dockerfile.HBase
new file mode 100644
index 0000000000..85c48e7da4
--- /dev/null
+++ b/build/Dockerfile.HBase
@@ -0,0 +1,319 @@
+#
+# 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.
+#
+
+# This Dockerfile is for building a HBase Cluster used in testing
+
+# Usage:
+#   Run the docker command below
+#      docker build \
+#        --build-arg APACHE_MIRROR="https://archive.apache.org/dist"; \
+#        --build-arg HBASE_VERSION="2.5.12" \
+#        --build-arg ZK_VERSION="3.8.4" \
+#        --file build/Dockerfile.HBase \
+#        --tag nekyuubi/kyuubi-hbase-cluster:<tag> \
+#        .
+#   Options:
+#     -f, --file  this docker file
+#     -t, --tag   the target repo and tag name
+#     more options can be found with -h, --help
+
+FROM eclipse-temurin:8-jdk-focal
+
+ENV DEBIAN_FRONTEND=noninteractive
+
+ARG HBASE_VERSION=2.5.12
+ARG APACHE_MIRROR=https://archive.apache.org/dist
+ARG ZK_VERSION=3.8.1
+
+ENV HBASE_HOME=/opt/hbase
+ENV ZK_HOME=/opt/zookeeper
+ENV JAVA_HOME=/opt/java/openjdk
+ENV PATH=$PATH:$HBASE_HOME/bin
+
+RUN apt-get update && apt-get install -y \
+    krb5-kdc \
+    krb5-admin-server \
+    wget \
+    busybox \
+    && mkdir /opt/busybox && \
+    busybox --install /opt/busybox
+
+# download hbase
+RUN wget -q 
${APACHE_MIRROR}/hbase/${HBASE_VERSION}/hbase-${HBASE_VERSION}-bin.tar.gz && \
+    tar -xzf hbase-${HBASE_VERSION}-bin.tar.gz -C /opt && \
+    mv /opt/hbase-${HBASE_VERSION} ${HBASE_HOME} && \
+    rm hbase-${HBASE_VERSION}-bin.tar.gz
+
+# download zookeeper
+RUN wget -q 
${APACHE_MIRROR}/zookeeper/zookeeper-${ZK_VERSION}/apache-zookeeper-${ZK_VERSION}-bin.tar.gz
 && \
+    tar -xzf apache-zookeeper-${ZK_VERSION}-bin.tar.gz -C /opt && \
+    mv /opt/apache-zookeeper-${ZK_VERSION}-bin ${ZK_HOME} && \
+    rm apache-zookeeper-${ZK_VERSION}-bin.tar.gz
+
+# add config files
+COPY <<"EOF" /etc/krb5.conf
+
+[libdefaults]
+    default_realm = TEST.ORG
+    dns_lookup_realm = false
+    dns_lookup_kdc = false
+
+[realms]
+    TEST.ORG = {
+        kdc = localhost:88
+    }
+
+EOF
+
+COPY <<"EOF" ${HBASE_HOME}/conf/hbase-site.xml
+
+<configuration>
+    <property>
+        <name>hbase.cluster.distributed</name>
+        <value>true</value>
+    </property>
+    <property>
+        <name>hbase.rootdir</name>
+        <value>file:///tmp/hbase-data</value>
+    </property>
+    <property>
+        <name>hbase.security.authentication</name>
+        <value>kerberos</value>
+    </property>
+    <property>
+        <name>hbase.master.kerberos.principal</name>
+        <value>hbase/[email protected]</value>
+    </property>
+    <property>
+        <name>hbase.master.keytab.file</name>
+        <value>/opt/hbase/conf/hbase.keytab</value>
+    </property>
+    <property>
+        <name>hbase.regionserver.kerberos.principal</name>
+        <value>hbase/[email protected]</value>
+    </property>
+    <property>
+        <name>hbase.regionserver.keytab.file</name>
+        <value>/opt/hbase/conf/hbase.keytab</value>
+    </property>
+    <property>
+        <name>hbase.security.authorization</name>
+        <value>true</value>
+    </property>
+    <property>
+        <name>hbase.master.hostname</name>
+        <value>localhost</value>
+    </property>
+    <property>
+        <name>hbase.regionserver.hostname</name>
+        <value>localhost</value>
+    </property>
+    <property>
+        <name>hbase.unsafe.regionserver.hostname</name>
+        <value>localhost</value>
+    </property>
+    <property>
+        <name>hbase.regionserver.ipc.address</name>
+        <value>0.0.0.0</value>
+    </property>
+    <property>
+        <name>hbase.master.ipc.address</name>
+        <value>0.0.0.0</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.quorum</name>
+        <value>localhost</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.property.clientPort</name>
+        <value>2181</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.client.keytab.file</name>
+        <value>/opt/hbase/conf/hbase.keytab</value>
+    </property>
+    <property>
+        <name>hbase.zookeeper.client.kerberos.principal</name>
+        <value>hbase/[email protected]</value>
+    </property>
+    <property>
+        <name>hbase.unsafe.stream.capability.enforce</name>
+        <value>false</value>
+    </property>
+    <property>
+        <name>hbase.coprocessor.region.classes</name>
+        
<value>org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController</value>
+    </property>
+    <property>
+        <name>hbase.coprocessor.master.classes</name>
+        <value>org.apache.hadoop.hbase.security.access.AccessController</value>
+    </property>
+</configuration>
+
+EOF
+
+COPY <<"EOF" ${HBASE_HOME}/conf/core-site.xml
+
+<configuration>
+    <property>
+        <name>hadoop.security.authentication</name>
+        <value>kerberos</value>
+    </property>
+    <property>
+        <name>hadoop.security.authorization</name>
+        <value>true</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.hbase.hosts</name>
+        <value>*</value>
+    </property>
+    <property>
+        <name>hadoop.proxyuser.hbase.groups</name>
+        <value>*</value>
+    </property>
+</configuration>
+
+EOF
+
+COPY <<"EOF" ${ZK_HOME}/conf/zk-jaas.conf
+Server {
+  com.sun.security.auth.module.Krb5LoginModule required
+  useKeyTab=true
+  storeKey=true
+  useTicketCache=false
+  keyTab="/opt/zookeeper/conf/zookeeper.keytab"
+  principal="zookeeper/[email protected]";
+};
+
+Client {
+   com.sun.security.auth.module.Krb5LoginModule required
+   useTicketCache=true;
+};
+EOF
+
+COPY <<"EOF" ${ZK_HOME}/conf/zoo.cfg
+tickTime=2000
+dataDir=/opt/zookeeper/data
+clientPort=2181
+# Kerberos settings
+authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
+kerberos.keytab.file=/opt/zookeeper/conf/zk.keytab
+kerberos.principal=zookeeper/[email protected]
+requireClientAuthScheme=sasl
+EOF
+
+COPY <<"EOF" /entrypoint.sh
+#!/bin/bash
+set -e
+
+# Function: Wait for a service to be available on specified ports
+# Parameters:
+#   $1 - Service name
+#   $2 - Timeout in seconds, default 60 seconds
+#   $3 - Check interval in seconds, default 1 second
+#   Remaining parameters - List of ports to check
+wait_for_service() {
+    local service_name="$1"
+    local timeout="${2:-60}"
+    local interval="${3:-1}"
+    shift 3  # Remove the first three parameters, remaining are ports
+
+    local end_time=$((SECONDS + timeout))
+    local ports_available=false
+    echo "Starting to wait for $service_name service to be available on ports 
$@..."
+
+    while [ $SECONDS -lt $end_time ]; do
+        ports_available=true
+
+        # Check all provided ports
+        for port in "$@"; do
+            if ! /opt/busybox/nc localhost "$port" -e true; then
+                ports_available=false
+                break
+            fi
+        done
+
+        if $ports_available; then
+            echo "$service_name service is now available"
+            return 0
+        else
+            echo "$service_name service is not available, retrying in 
$interval seconds..."
+            sleep $interval
+        fi
+    done
+
+    echo "Timeout reached. $service_name service is still not available."
+    exit 1
+}
+
+export REALM="TEST.ORG"
+export KADMIN_PASS="admin"
+export HBASE_CONF_DIR="${HBASE_HOME}/conf"
+export ZK_CONF_DIR="${ZK_HOME}/conf"
+
+echo "Creating KDC database..."
+kdb5_util create -r ${REALM} -s -P ${KADMIN_PASS}
+
+echo "Creating principals and keytabs..."
+kadmin.local -q "addprinc -randkey hbase/localhost@${REALM}"
+kadmin.local -q "ktadd -k ${HBASE_CONF_DIR}/hbase.keytab 
hbase/localhost@${REALM}"
+
+kadmin.local -q "addprinc -randkey zookeeper/localhost@${REALM}"
+kadmin.local -q "ktadd -k ${ZK_CONF_DIR}/zookeeper.keytab 
zookeeper/localhost@${REALM}"
+
+mkdir -p ${HBASE_HOME}/logs/
+mkdir -p ${ZK_HOME}/logs/
+
+echo "Principals created. Starting supervisor..."
+# start kdc service
+echo "Start kdc ..."
+/etc/init.d/krb5-kdc start
+wait_for_service "kdc" 15 1 88
+
+# start zookeeper server
+export 
JVMFLAGS="-Djava.security.auth.login.config=/opt/zookeeper/conf/zk-jaas.conf"
+/opt/zookeeper/bin/zkServer.sh start
+unset JVMFLAGS
+wait_for_service "zookeeper" 15 1 2181
+
+kinit -kt ${HBASE_CONF_DIR}/hbase.keytab hbase/localhost@${REALM}
+
+# start hbase master server
+nohup /opt/hbase/bin/hbase master start  > "/opt/hbase/logs/master.log" 2>&1 < 
/dev/null &
+wait_for_service "master" 15 1 16000
+
+# start hbase region server
+nohup /opt/hbase/bin/hbase regionserver start  > 
"/opt/hbase/logs/regionserver.log" 2>&1 < /dev/null &
+wait_for_service "master" 15 1 16020
+
+tail -F --follow=name --retry -v  /opt/zookeeper/logs/* /opt/hbase/logs/*
+EOF
+
+RUN chmod +x /entrypoint.sh
+
+# HBase Master UI
+EXPOSE 16010
+# Kerberos
+EXPOSE 88/udp
+# zookeeper
+EXPOSE 2181
+# HBase Master
+EXPOSE 16000
+# HBase RegionServer
+EXPOSE 16020
+
+ENTRYPOINT ["/entrypoint.sh"]
diff --git a/kyuubi-server/pom.xml b/kyuubi-server/pom.xml
index 303d144106..6716921931 100644
--- a/kyuubi-server/pom.xml
+++ b/kyuubi-server/pom.xml
@@ -123,6 +123,11 @@
             <version>${kyuubi-relocated.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-shaded-client-byo-hadoop</artifactId>
+        </dependency>
+
         <dependency>
             <groupId>org.antlr</groupId>
             <artifactId>antlr4-runtime</artifactId>
diff --git 
a/kyuubi-server/src/main/resources/META-INF/services/org.apache.kyuubi.credentials.HadoopDelegationTokenProvider
 
b/kyuubi-server/src/main/resources/META-INF/services/org.apache.kyuubi.credentials.HadoopDelegationTokenProvider
index 95d6e1987f..f490c22447 100644
--- 
a/kyuubi-server/src/main/resources/META-INF/services/org.apache.kyuubi.credentials.HadoopDelegationTokenProvider
+++ 
b/kyuubi-server/src/main/resources/META-INF/services/org.apache.kyuubi.credentials.HadoopDelegationTokenProvider
@@ -15,5 +15,6 @@
 # limitations under the License.
 #
 
+org.apache.kyuubi.credentials.HBaseDelegationTokenProvider
 org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider
 org.apache.kyuubi.credentials.HiveDelegationTokenProvider
diff --git 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProvider.scala
 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProvider.scala
new file mode 100644
index 0000000000..cac9305d7e
--- /dev/null
+++ 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProvider.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.kyuubi.credentials
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.hbase.client.{Connection, ConnectionFactory}
+import org.apache.hadoop.hbase.security.token.ClientTokenUtil
+import org.apache.hadoop.security.Credentials
+
+import org.apache.kyuubi.{KyuubiException, Logging}
+import org.apache.kyuubi.config.KyuubiConf
+import 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider.doAsProxyUser
+
+private class HBaseDelegationTokenProvider
+  extends HadoopDelegationTokenProvider with Logging {
+
+  override def serviceName: String = "hbase"
+  private var tokenRequired: Boolean = _
+  private var hbaseConf: Configuration = _
+  private var kyuubiConf: KyuubiConf = _
+
+  override def initialize(hadoopConf: Configuration, kyuubiConf: KyuubiConf): 
Unit = {
+    this.kyuubiConf = kyuubiConf
+    this.hbaseConf = hadoopConf
+    this.tokenRequired =
+      hbaseConf.get("hbase.security.authentication", "simple").toLowerCase() 
== "kerberos"
+  }
+
+  override def obtainDelegationTokens(
+      owner: String,
+      creds: Credentials): Unit = {
+    doAsProxyUser(owner) {
+      var conn: Connection = null
+      try {
+        info(s"Getting HBase delegation token for ${owner} ...")
+        conn = ConnectionFactory.createConnection(hbaseConf)
+        val token = ClientTokenUtil.obtainToken(conn)
+        info(s"Get HBase delegation token ${token}")
+        creds.addToken(token.getService, token)
+      } catch {
+        case e: Throwable =>
+          throw new KyuubiException(s"Failed to get HBase delegation token 
owned by $owner", e)
+      } finally {
+        if (conn != null) {
+          conn.close()
+        }
+      }
+    }
+  }
+
+  override def delegationTokensRequired(): Boolean = {
+    tokenRequired
+  }
+}
diff --git 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/WithSecuredHBaseContainer.scala
 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/WithSecuredHBaseContainer.scala
new file mode 100644
index 0000000000..970388701a
--- /dev/null
+++ 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/WithSecuredHBaseContainer.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.kyuubi
+
+import java.io.File
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Paths}
+
+import com.dimafeng.testcontainers.{ContainerDef, 
FixedHostPortGenericContainer}
+import com.dimafeng.testcontainers.scalatest.TestContainerForAll
+import com.github.dockerjava.api.model.{ExposedPort, Ports}
+import org.apache.hadoop.conf.Configuration
+import org.testcontainers.containers.wait.strategy.HostPortWaitStrategy
+
+trait WithSecuredHBaseContainer extends KerberizedTestHelper with 
TestContainerForAll {
+
+  final val HADOOP_SECURITY_AUTHENTICATION = "kerberos"
+  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 = "nekyuubi/kyuubi-hbase-cluster:latest"
+
+  private val tempDir = Utils.createTempDir(prefix = "kyuubi-server-hbase")
+  private val exposedKdcPort = 88
+  private val exposedZkPort = 2181
+  private val exposedHbaseMasterPort = 16000
+  private val exposedHbaseRegionServerPort = 16020
+  private val testPrincipalOverride =
+    HBASE_KERBEROS_PRINCIPAL + "@" + HBASE_KERBEROS_REALM
+  private val krb5ConfPathOverride = new File(tempDir.toFile, 
"krb5.conf").getAbsolutePath
+  val hbaseConf: Configuration = new Configuration(false)
+
+  override val testKeytab: String = new File(tempDir.toFile, 
"hbase.service.keytab").getAbsolutePath
+  override val containerDef: HBaseContainer.Def = HBaseContainer.Def(
+    DOCKER_IMAGE_NAME,
+    exposedKdcPort,
+    exposedHbaseMasterPort,
+    exposedHbaseRegionServerPort,
+    exposedZkPort)
+
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    testPrincipal = testPrincipalOverride
+    krb5ConfPath = krb5ConfPathOverride
+  }
+
+  override def afterContainersStart(containers: Containers): Unit = {
+    containers.copyFileFromContainer(HBASE_KERBEROS_KEYTAB, testKeytab)
+    hbaseConf.set("hbase.zookeeper.quorum", "localhost:" + 
containers.mappedPort(exposedZkPort))
+    hbaseConf.set("hbase.security.authentication", "kerberos")
+    hbaseConf.set("hadoop.security.authentication", "kerberos")
+    hbaseConf.set(
+      "hbase.master.kerberos.principal",
+      HBASE_KERBEROS_PRINCIPAL + "@" + HBASE_KERBEROS_REALM)
+    hbaseConf.set(
+      "hbase.regionserver.kerberos.principal",
+      HBASE_KERBEROS_PRINCIPAL + "@" + HBASE_KERBEROS_REALM)
+
+    val krb5ConfContent =
+      s"""[libdefaults]
+         |  default_realm = $HBASE_KERBEROS_REALM
+         |  dns_lookup_realm = false
+         |  dns_lookup_kdc = false
+         |  forwardable = true
+         |
+         |[realms]
+         |  ${HBASE_KERBEROS_REALM} = {
+         |    kdc = localhost:${containers.getKdcUdpPort}
+         |  }
+         |
+         |""".stripMargin
+
+    val writer = Files.newBufferedWriter(Paths.get(krb5ConfPathOverride), 
StandardCharsets.UTF_8)
+    writer.write(krb5ConfContent)
+    writer.close()
+  }
+}
+
+class HBaseContainer(exposedKdcPort: Int, dockerImage: String)
+  extends FixedHostPortGenericContainer(dockerImage) {
+
+  def getKdcUdpPort: Int = {
+    container.getContainerInfo.getNetworkSettings.getPorts.getBindings
+      .get(ExposedPort.udp(exposedKdcPort)).head.getHostPortSpec.toInt
+  }
+}
+
+object HBaseContainer {
+  case class Def(
+      dockerImage: String,
+      exposedKdcPort: Int,
+      exposedHbaseMasterPort: Int,
+      exposedHbaseRegionServerPort: Int,
+      exposedZkPort: Int,
+      env: Map[String, String] = Map())
+    extends ContainerDef {
+
+    override type Container = HBaseContainer
+
+    override def createContainer(): Container = {
+      val container = new HBaseContainer(
+        exposedKdcPort,
+        dockerImage)
+
+      container.container.withExposedPorts(
+        exposedKdcPort,
+        exposedZkPort)
+      // The reason for using a fixed port here is that the addresses of the 
HBase Region Server
+      // and HBase Master Server are recorded in ZooKeeper and cannot be 
obtained through the
+      // method of container-mapped ports.
+      container.container.withFixedExposedPort(
+        exposedHbaseRegionServerPort,
+        exposedHbaseRegionServerPort)
+      container.container.withFixedExposedPort(exposedHbaseMasterPort, 
exposedHbaseMasterPort)
+      container.container.setWaitStrategy(new HostPortWaitStrategy()
+        .forPorts(exposedHbaseMasterPort, exposedHbaseRegionServerPort, 
exposedZkPort))
+
+      container.container.withCreateContainerCmdModifier(cmd => {
+        val udpExposedPort = ExposedPort.udp(exposedKdcPort)
+        val exposedPorts = new java.util.LinkedList[ExposedPort]()
+        for (p <- cmd.getExposedPorts) {
+          exposedPorts.add(p)
+        }
+        exposedPorts.add(udpExposedPort)
+        cmd.withExposedPorts(exposedPorts)
+
+        // Add previous port bindings and UDP port binding
+        val ports = cmd.getHostConfig.getPortBindings
+        ports.bind(udpExposedPort, Ports.Binding.empty)
+        cmd.getHostConfig.withPortBindings(ports)
+      })
+      container
+    }
+  }
+}
diff --git 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProviderSuite.scala
 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProviderSuite.scala
new file mode 100644
index 0000000000..cf4ba67a93
--- /dev/null
+++ 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/credentials/HBaseDelegationTokenProviderSuite.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.kyuubi.credentials
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.hbase.security.token.AuthenticationTokenIdentifier
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.security.{Credentials, UserGroupInformation}
+
+import org.apache.kyuubi.WithSecuredHBaseContainer
+import org.apache.kyuubi.config.KyuubiConf
+
+class HBaseDelegationTokenProviderSuite extends WithSecuredHBaseContainer {
+
+  test("obtain hbase delegation token") {
+    tryWithSecurityEnabled {
+      UserGroupInformation.loginUserFromKeytab(testPrincipal, testKeytab)
+
+      val kyuubiConf = new KyuubiConf(false)
+      val provider = new HBaseDelegationTokenProvider()
+      provider.initialize(hbaseConf, kyuubiConf)
+      assert(provider.delegationTokensRequired())
+
+      val owner = "who"
+      val credentials = new Credentials
+      provider.obtainDelegationTokens(owner, credentials)
+
+      val aliasAndToken =
+        credentials.getTokenMap.asScala
+          .head
+      assert(aliasAndToken._2 != null)
+
+      val token = aliasAndToken._2
+      val tokenIdent = 
token.decodeIdentifier().asInstanceOf[AuthenticationTokenIdentifier]
+      assertResult(new Text("HBASE_AUTH_TOKEN"))(token.getKind)
+      assertResult(owner)(tokenIdent.getUsername)
+    }
+  }
+}
diff --git a/pom.xml b/pom.xml
index 8efc5bf956..5050ebbe6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -197,6 +197,8 @@
         <slf4j.version>1.7.36</slf4j.version>
         <clickhouse-java.version>0.6.5</clickhouse-java.version>
         <snakeyaml.version>2.2</snakeyaml.version>
+        <hbase.version>2.5.12-hadoop3</hbase.version>
+        <hbase.deps.scope>provided</hbase.deps.scope>
         <!--
           DO NOT forget to change the following properties when change the 
minor version of Spark:
           `delta.version`, `delta.artifact`, 
`maven.plugin.scalatest.exclude.tags`
@@ -1264,6 +1266,19 @@
                     </exclusion>
                 </exclusions>
             </dependency>
+
+            <dependency>
+                <groupId>org.apache.hbase</groupId>
+                <artifactId>hbase-shaded-client-byo-hadoop</artifactId>
+                <version>${hbase.version}</version>
+                <scope>${hbase.deps.scope}</scope>
+                <exclusions>
+                    <exclusion>
+                        <groupId>org.apache.yetus</groupId>
+                        <artifactId>audience-annotations</artifactId>
+                    </exclusion>
+                </exclusions>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 

Reply via email to