SeongHoon Ku created HIVE-28913:
-----------------------------------

             Summary: When calling the reconnect() method in 
HiveMetaStoreClient, if HADOOP_PROXY_USER is set, a delegation token must be 
issued for the proxy user.
                 Key: HIVE-28913
                 URL: https://issues.apache.org/jira/browse/HIVE-28913
             Project: Hive
          Issue Type: Bug
          Components: Metastore
         Environment: hive 2.3.x
            Reporter: SeongHoon Ku
            Assignee: SeongHoon Ku


Hello.

I am currently developing a Kafka Connector that uses Iceberg as a sink. We are 
using Hive version 2.3.x.

In our Kafka Connect setup, there is a user configured with proxy privileges. 
This setup allows this user to access Iceberg tables (which use Hive Metastore 
as the catalog) under the permissions of other users by impersonating them.

Consequently, the HADOOP_PROXY_USER environment variable is set for each 
connector instance, causing the HiveMetaStoreClient to be initialized using the 
specified proxy user identity.

The problem arises when the Hive Metastore is restarted (e.g., due to 
maintenance or an operation). In this scenario, the reconnect() method of the 
HiveMetaStoreClient, which is part of the Iceberg API, fails.

As you can see in the stack trace below, the initial connection attempt to the 
restarted Hive Metastore fails. Then, during the execution of the connection 
retry logic, a GSSException occurs.

The reconnect() method within HiveMetaStoreClient should include logic to 
obtain a delegation token for the proxy user specifically when the 
HADOOP_PROXY_USER environment variable is set.

Although we are using the Hive 2.3.x Metastore, this required logic seems to be 
missing even in the current master branch. Therefore, I am reporting this issue.

{code}
[2025-04-19 15:45:48,446] INFO [45337-limtan-ib-g19-3|task-0] Trying to connect 
to metastore with URI thrift://koodin-test-metastore-1.com:9083 
(hive.metastore:410)
[2025-04-19 15:45:48,452] ERROR [45337-limtan-ib-g19-3|task-0] SASL negotiation 
failure (org.apache.thrift.transport.TSaslTransport:278)
javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: 
No valid credentials provided (Mechanism level: Failed to find any Kerberos 
tgt)]
        at 
jdk.security.jgss/com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(Unknown
 Source)
        at 
org.apache.thrift.transport.TSaslClientTransport.handleSaslStartMessage(TSaslClientTransport.java:96)
        at 
org.apache.thrift.transport.TSaslTransport.open(TSaslTransport.java:236)
        at 
org.apache.thrift.transport.TSaslClientTransport.open(TSaslClientTransport.java:39)
        at 
org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport$1.run(TUGIAssumingTransport.java:52)
        at 
org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport$1.run(TUGIAssumingTransport.java:49)
        at java.base/java.security.AccessController.doPrivileged(Unknown Source)
        at java.base/javax.security.auth.Subject.doAs(Unknown Source)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
        at 
org.apache.hadoop.hive.thrift.client.TUGIAssumingTransport.open(TUGIAssumingTransport.java:49)
        at 
org.apache.hadoop.hive.metastore.HiveMetaStoreClient.open(HiveMetaStoreClient.java:494)
        at 
org.apache.hadoop.hive.metastore.HiveMetaStoreClient.reconnect(HiveMetaStoreClient.java:341)
        at 
org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:163)
        at jdk.proxy4/jdk.proxy4.$Proxy188.getTable(Unknown Source)
        at 
org.apache.iceberg.hive.HiveTableOperations.lambda$doRefresh$0(HiveTableOperations.java:146)
        at org.apache.iceberg.ClientPoolImpl.run(ClientPoolImpl.java:58)
        at org.apache.iceberg.ClientPoolImpl.run(ClientPoolImpl.java:51)
        at 
org.apache.iceberg.hive.CachedClientPool.run(CachedClientPool.java:122)
        at 
org.apache.iceberg.hive.HiveTableOperations.doRefresh(HiveTableOperations.java:146)
        at 
org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:97)
        at org.apache.iceberg.BaseTable.refresh(BaseTable.java:73)
        at io.tabular.iceberg.connect.channel.Worker.write(Worker.java:133)
        at io.tabular.iceberg.connect.channel.TaskImpl.put(TaskImpl.java:51)
        at 
io.tabular.iceberg.connect.IcebergSinkTask.lambda$put$4(IcebergSinkTask.java:181)
        at java.base/java.security.AccessController.doPrivileged(Unknown Source)
        at java.base/javax.security.auth.Subject.doAs(Unknown Source)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
        at 
io.tabular.iceberg.connect.IcebergSinkTask.put(IcebergSinkTask.java:179)
        at 
org.apache.kafka.connect.runtime.WorkerSinkTask.deliverMessages(WorkerSinkTask.java:605)
        at 
org.apache.kafka.connect.runtime.WorkerSinkTask.poll(WorkerSinkTask.java:344)
        at 
org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:246)
        at 
org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:215)
        at 
org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:225)
        at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:280)
        at 
org.apache.kafka.connect.runtime.isolation.Plugins.lambda$withClassLoader$1(Plugins.java:237)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
        at java.base/java.util.concurrent.FutureTask.run(Unknown Source)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown 
Source)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown 
Source)
        at java.base/java.lang.Thread.run(Unknown Source)
Caused by: GSSException: No valid credentials provided (Mechanism level: Failed 
to find any Kerberos tgt)
        at 
java.security.jgss/sun.security.jgss.krb5.Krb5InitCredential.getInstance(Unknown
 Source)
        at 
java.security.jgss/sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Unknown
 Source)
        at 
java.security.jgss/sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Unknown
 Source)
        at 
java.security.jgss/sun.security.jgss.GSSManagerImpl.getMechanismContext(Unknown 
Source)
        at 
java.security.jgss/sun.security.jgss.GSSContextImpl.initSecContext(Unknown 
Source)
        at 
java.security.jgss/sun.security.jgss.GSSContextImpl.initSecContext(Unknown 
Source)
        ... 40 more
{code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to