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

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


The following commit(s) were added to refs/heads/master by this push:
     new d45421b  [KYUUBI #1008][FOLLOWUP] Can not obtain hadoop file system 
delegation token
d45421b is described below

commit d45421b37e87c9884b8c4daa2f03bcff3087d497
Author: zhouyifan279 <[email protected]>
AuthorDate: Tue Sep 28 09:38:59 2021 +0800

    [KYUUBI #1008][FOLLOWUP] Can not obtain hadoop file system delegation token
    
    ### _Why are the changes needed?_
    Kyuubi server throws exception when obtaining hadoop file system delegation 
token
    
    ```
    21/09/27 19:53:07 WARN credentials.HadoopCredentialsManager: Failed to 
update tokens for morty, try again in 3000 ms
    java.io.IOException: DestHost:destPort kdc.hadoop.com:9000 , 
LocalHost:localPort client.hadoop.com/192.168.56.1:0. Failed on local 
exception: java.io.IOException: Couldn't set up IO streams: 
java.lang.IllegalArgumentException: Failed to specify server's Kerberos 
principal n
    ame
            at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native 
Method)
            at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
            at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
            at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
            at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:836)
            at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:811)
            at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1566)
            at org.apache.hadoop.ipc.Client.call(Client.java:1508)
            at org.apache.hadoop.ipc.Client.call(Client.java:1405)
            at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
            at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
            at com.sun.proxy.$Proxy10.getDelegationToken(Unknown Source)
            at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getDelegationToken(ClientNamenodeProtocolTranslatorPB.java:1081)
            at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
            at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            at java.lang.reflect.Method.invoke(Method.java:498)
            at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:422)
            at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
            at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
            at 
org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
            at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
            at com.sun.proxy.$Proxy11.getDelegationToken(Unknown Source)
            at 
org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:703)
            at 
org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:1845)
            at 
org.apache.hadoop.security.token.DelegationTokenIssuer.collectDelegationTokens(DelegationTokenIssuer.java:95)
            at 
org.apache.hadoop.security.token.DelegationTokenIssuer.addDelegationTokens(DelegationTokenIssuer.java:76)
            at 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider.$anonfun$obtainDelegationTokens$2(HadoopFsDelegationTokenProvider.scala:66)
            at scala.collection.immutable.Set$Set1.foreach(Set.scala:141)
            at 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider.$anonfun$obtainDelegationTokens$1(HadoopFsDelegationTokenProvider.scala:64)
            at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
            at 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider$$anon$1.run(HadoopFsDelegationTokenProvider.scala:106)
            at java.security.AccessController.doPrivileged(Native Method)
            at javax.security.auth.Subject.doAs(Subject.java:422)
            at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
            at 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider$.doAsProxyUser(HadoopFsDelegationTokenProvider.scala:105)
            at 
org.apache.kyuubi.credentials.HadoopFsDelegationTokenProvider.obtainDelegationTokens(HadoopFsDelegationTokenProvider.scala:57)
            at 
org.apache.kyuubi.credentials.HadoopCredentialsManager$$anon$1.$anonfun$run$1(HadoopCredentialsManager.scala:210)
            at 
org.apache.kyuubi.credentials.HadoopCredentialsManager$$anon$1.$anonfun$run$1$adapted(HadoopCredentialsManager.scala:210)
            at scala.collection.Iterator.foreach(Iterator.scala:943)
            at scala.collection.Iterator.foreach$(Iterator.scala:943)
            at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
            at 
scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:214)
            at 
org.apache.kyuubi.credentials.HadoopCredentialsManager$$anon$1.run(HadoopCredentialsManager.scala:210)
            at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
            at java.util.concurrent.FutureTask.run(FutureTask.java:266)
            at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
            at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
            at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
            at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
            at java.lang.Thread.run(Thread.java:748)
    Caused by: java.io.IOException: Couldn't set up IO streams: 
java.lang.IllegalArgumentException: Failed to specify server's Kerberos 
principal name
            at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:884)
            at 
org.apache.hadoop.ipc.Client$Connection.access$3800(Client.java:413)
            at org.apache.hadoop.ipc.Client.getConnection(Client.java:1636)
            at org.apache.hadoop.ipc.Client.call(Client.java:1452)
            ... 43 more
    Caused by: java.lang.IllegalArgumentException: Failed to specify server's 
Kerberos principal name
            at 
org.apache.hadoop.security.SaslRpcClient.getServerPrincipal(SaslRpcClient.java:327)
            at 
org.apache.hadoop.security.SaslRpcClient.createSaslClient(SaslRpcClient.java:234)
            at 
org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:160)
            at 
org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:390)
            at 
org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:622)
            at 
org.apache.hadoop.ipc.Client$Connection.access$2300(Client.java:413)
            at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:822)
            at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:818)
            at java.security.AccessController.doPrivileged(Native Method)
            at javax.security.auth.Subject.doAs(Subject.java:422)
            at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1762)
            at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:818)
            ... 46 more
    ```
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including 
negative and positive cases if possible
    
    - [x] Add screenshots for manual tests if appropriate
    
![image](https://user-images.githubusercontent.com/88070094/134904531-faac4be0-9fe4-4c4a-8ab8-70aa5ea5fb54.png)
    
    - [x] [Run 
test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests)
 locally before make a pull request
    
    Closes #1166 from zhouyifan279/#1008.
    
    Closes #1008
    
    43dbf605 [zhouyifan279] [KYUUBI #1008][FOLLOWUP] Can not obtain hadoop fs 
delegation token
    
    Authored-by: zhouyifan279 <[email protected]>
    Signed-off-by: ulysses-you <[email protected]>
---
 .../apache/kyuubi/credentials/HadoopFsDelegationTokenProvider.scala    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopFsDelegationTokenProvider.scala
 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopFsDelegationTokenProvider.scala
index 148b084..2f6b17a 100644
--- 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopFsDelegationTokenProvider.scala
+++ 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/credentials/HadoopFsDelegationTokenProvider.scala
@@ -24,6 +24,7 @@ import scala.collection.JavaConverters._
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hdfs.HdfsConfiguration
 import org.apache.hadoop.security.{Credentials, SecurityUtil, 
UserGroupInformation}
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod
 
@@ -47,7 +48,7 @@ class HadoopFsDelegationTokenProvider extends 
HadoopDelegationTokenProvider with
     // The UserGroupInformation object used to create it is part of that key.
     // If cache is enabled, new FileSystem objects are created and cached at 
every method
     // invocation.
-    this.hadoopConf = disableFsCache(kyuubiConf, hadoopConf)
+    this.hadoopConf = disableFsCache(kyuubiConf, new 
HdfsConfiguration(hadoopConf))
     this.kyuubiConf = kyuubiConf
   }
 

Reply via email to