[ 
https://issues.apache.org/jira/browse/FLINK-6376?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16000162#comment-16000162
 ] 

ASF GitHub Bot commented on FLINK-6376:
---------------------------------------

Github user Rucongzhang commented on the issue:

    https://github.com/apache/flink/pull/3776
  
    After resolving this problem, we find another problem, when we configure 
the keytab 、principal, and  add the HDFS delegation token,  the JM 、TM also use 
this token, but not keytab when communication with HDFS. When token is expired, 
no one in flink to refresh the token.  
    But the purpose of adding this token , which is only used for  yarn node 
manager. We now is resolving this problem. Thanks!


> when deploy flink cluster on the yarn, it is lack of hdfs delegation token.
> ---------------------------------------------------------------------------
>
>                 Key: FLINK-6376
>                 URL: https://issues.apache.org/jira/browse/FLINK-6376
>             Project: Flink
>          Issue Type: Bug
>          Components: Security, YARN
>            Reporter: zhangrucong1982
>            Assignee: zhangrucong1982
>
> 1、I use the flink of version 1.2.0. And  I deploy the flink cluster on the 
> yarn. The hadoop version is 2.7.2.
> 2、I use flink in security model with the keytab and principal. And the key 
> configuration is :security.kerberos.login.keytab: /home/ketab/test.keytab 
> 、security.kerberos.login.principal: test.
> 3、The yarn configuration is default and enable the yarn log aggregation 
> configuration" yarn.log-aggregation-enable : true";
> 4、 Deploying the flink cluster  on the yarn,  the yarn Node manager occur the 
> following failure when aggregation the log in HDFS. The basic reason is lack 
> of HDFS  delegation token. 
>  java.io.IOException: Failed on local exception: java.io.IOException: 
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate 
> via:[TOKEN, KERBEROS]; Host Details : local host is: 
> "SZV1000258954/10.162.181.24"; destination host is: "SZV1000258954":25000;
>         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:796)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1515)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1447)
>         at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
>         at com.sun.proxy.$Proxy26.getFileInfo(Unknown Source)
>         at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:802)
>         at sun.reflect.GeneratedMethodAccessor17.invoke(Unknown Source)
>         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:201)
>         at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:103)
>         at com.sun.proxy.$Proxy27.getFileInfo(Unknown Source)
>         at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1919)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1500)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem$27.doCall(DistributedFileSystem.java:1496)
>         at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>         at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1496)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.checkExists(LogAggregationService.java:271)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.access$100(LogAggregationService.java:68)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService$1.run(LogAggregationService.java:299)
>         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:1769)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.createAppDir(LogAggregationService.java:284)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.initAppAggregator(LogAggregationService.java:390)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.initApp(LogAggregationService.java:342)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.handle(LogAggregationService.java:470)
>         at 
> org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.LogAggregationService.handle(LogAggregationService.java:68)
>         at 
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:194)
>         at 
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:120)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: 
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate 
> via:[TOKEN, KERBEROS]
>         at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:722)
>         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:1769)
>         at 
> org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:685)
>         at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:772)
>         at 
> org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:394)
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1564)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1486)
>         ... 29 more
> Caused by: org.apache.hadoop.security.AccessControlException: Client cannot 
> authenticate via:[TOKEN, KERBEROS]
>         at 
> org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:177)
>         at 
> org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:404)
>         at 
> org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:581)
>         at 
> org.apache.hadoop.ipc.Client$Connection.access$1900(Client.java:394)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:764)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:760)
>         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:1769)
>         at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:759)
>         ... 32 more
> 5、the hadoop fix the hadoop issue 
> 14116(https://issues.apache.org/jira/browse/HADOOP-14116), if there is no 
> HDFS  delegation token, it will try 20 times after sleeping 1 second. So it 
> will cause the flink cluster deploy on yarn is very slowly, it will spent 
> about 5 minutes to deploy the cluster with 2 taskmanagers.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to