Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-6376

when deploy flink cluster on the yarn, it is lack of hdfs delegation token.

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 1.4.0, 1.3.2
    • Component/s: Security, YARN
    • Labels:
      None

      Description

      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.

        Issue Links

          Activity

          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user Rucongzhang opened a pull request:

          https://github.com/apache/flink/pull/3776

          FLINK-6376when deploy flink cluster on the yarn, it is lack of hdfs delegation.

          Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
          If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
          In addition to going through the list, please provide a meaningful description of your changes.

          • [ ] General
          • The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
          • The pull request addresses only one issue
          • Each commit in the PR has a meaningful commit message (including the JIRA id)
          • [ ] Documentation
          • Documentation has been added for new functionality
          • Old documentation affected by the pull request has been updated
          • JavaDoc for public methods has been added
          • [ ] Tests & Build
          • Functionality added by the pull request is covered by tests
          • `mvn clean verify` has been executed successfully locally or a Travis build has passed

          You can merge this pull request into a Git repository by running:

          $ git pull https://github.com/Rucongzhang/flink flink-6376

          Alternatively you can review and apply these changes as the patch at:

          https://github.com/apache/flink/pull/3776.patch

          To close this pull request, make a commit to your master/trunk branch
          with (at least) the following in the commit message:

          This closes #3776


          commit 256f519cda73571c73914c98b3c9ff4381520907
          Author: z00376786 <zhangrucong@huawei.com>
          Date: 2017-04-26T03:36:43Z

          when deploy flink cluster on the yarn, it is lack of hdfs delegation token


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user Rucongzhang opened a pull request: https://github.com/apache/flink/pull/3776 FLINK-6376 when deploy flink cluster on the yarn, it is lack of hdfs delegation. Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration. If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide] ( http://flink.apache.org/how-to-contribute.html ). In addition to going through the list, please provide a meaningful description of your changes. [ ] General The pull request references the related JIRA issue (" [FLINK-XXX] Jira title text") The pull request addresses only one issue Each commit in the PR has a meaningful commit message (including the JIRA id) [ ] Documentation Documentation has been added for new functionality Old documentation affected by the pull request has been updated JavaDoc for public methods has been added [ ] Tests & Build Functionality added by the pull request is covered by tests `mvn clean verify` has been executed successfully locally or a Travis build has passed You can merge this pull request into a Git repository by running: $ git pull https://github.com/Rucongzhang/flink flink-6376 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3776.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #3776 commit 256f519cda73571c73914c98b3c9ff4381520907 Author: z00376786 <zhangrucong@huawei.com> Date: 2017-04-26T03:36:43Z when deploy flink cluster on the yarn, it is lack of hdfs delegation token
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3776

          Looks like a good fix.

          I think it would be good to add secure yarn tests (IT Cases) that test this behavior. Otherwise it may soon be accidentally broken again...

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3776 Looks like a good fix. I think it would be good to add secure yarn tests (IT Cases) that test this behavior. Otherwise it may soon be accidentally broken again...
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , yes, i agree with you! I will see how to add the Yarn IT case.

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , yes, i agree with you! I will see how to add the Yarn IT case.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , when I run the YARNHighAvailabilityITCase, occur the following error. I set the flie /etc/hosts as following:
          9.96.101.32 9-96-101-32
          127.0.0.1 localhost

          The hadoop version I use is master default version 2.7.0. Do you know how to fix the following error?Thanks a lot in advance!

          Test testMultipleAMKill(org.apache.flink.yarn.YARNHighAvailabilityITCase) failed with:
          java.net.UnknownHostException: Invalid host name: local host is: (unknown); destination host is: "9-96-101-32":8032; java.net.UnknownHostException; For more details see: http://wiki.apache.org/hadoop/UnknownHost
          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:783)
          at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:742)
          at org.apache.hadoop.ipc.Client$Connection.<init>(Client.java:400)
          at org.apache.hadoop.ipc.Client.getConnection(Client.java:1448)
          at org.apache.hadoop.ipc.Client.call(Client.java:1377)
          at org.apache.hadoop.ipc.Client.call(Client.java:1359)
          at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
          at com.sun.proxy.$Proxy76.getApplications(Unknown Source)
          at org.apache.hadoop.yarn.api.impl.pb.client.ApplicationClientProtocolPBClientImpl.getApplications(ApplicationClientProtocolPBClientImpl.java:197)
          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:186)
          at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
          at com.sun.proxy.$Proxy77.getApplications(Unknown Source)
          at org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.getApplications(YarnClientImpl.java:285)
          at org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.getApplications(YarnClientImpl.java:262)
          at org.apache.flink.yarn.YarnTestBase.checkClusterEmpty(YarnTestBase.java:194)
          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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
          at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
          at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
          at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
          at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
          at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
          at org.junit.rules.RunRules.evaluate(RunRules.java:20)
          at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
          at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
          at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
          at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
          at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
          at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
          at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
          at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
          at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
          at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
          at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
          at org.junit.rules.RunRules.evaluate(RunRules.java:20)
          at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
          at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
          at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
          at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:51)
          at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:237)
          at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
          Caused by: java.net.UnknownHostException
          ... 47 more

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , when I run the YARNHighAvailabilityITCase, occur the following error. I set the flie /etc/hosts as following: 9.96.101.32 9-96-101-32 127.0.0.1 localhost The hadoop version I use is master default version 2.7.0. Do you know how to fix the following error?Thanks a lot in advance! Test testMultipleAMKill(org.apache.flink.yarn.YARNHighAvailabilityITCase) failed with: java.net.UnknownHostException: Invalid host name: local host is: (unknown); destination host is: "9-96-101-32":8032; java.net.UnknownHostException; For more details see: http://wiki.apache.org/hadoop/UnknownHost 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:783) at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:742) at org.apache.hadoop.ipc.Client$Connection.<init>(Client.java:400) at org.apache.hadoop.ipc.Client.getConnection(Client.java:1448) at org.apache.hadoop.ipc.Client.call(Client.java:1377) at org.apache.hadoop.ipc.Client.call(Client.java:1359) at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206) at com.sun.proxy.$Proxy76.getApplications(Unknown Source) at org.apache.hadoop.yarn.api.impl.pb.client.ApplicationClientProtocolPBClientImpl.getApplications(ApplicationClientProtocolPBClientImpl.java:197) 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:186) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102) at com.sun.proxy.$Proxy77.getApplications(Unknown Source) at org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.getApplications(YarnClientImpl.java:285) at org.apache.hadoop.yarn.client.api.impl.YarnClientImpl.getApplications(YarnClientImpl.java:262) at org.apache.flink.yarn.YarnTestBase.checkClusterEmpty(YarnTestBase.java:194) 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.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48) at org.junit.rules.RunRules.evaluate(RunRules.java:20) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68) at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:51) at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:237) at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70) Caused by: java.net.UnknownHostException ... 47 more
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3776

          The Yarn tests cannot with with Hadoop 2.3.0, which is the default version of master.

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3776 The Yarn tests cannot with with Hadoop 2.3.0, which is the default version of master.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , you mean the reason of this problem is the version of hadoop? The version of 2.7.2 is ok? Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , you mean the reason of this problem is the version of hadoop? The version of 2.7.2 is ok? Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , I can run the yarn IT case! Thanks very much. But, in the Yarn IT case:
          1、 It uses the yarn mini cluster, which is for testing.I do not know whether it is using HDFS Delegation token or not.
          2、And what's more, the HDFS Delegation Token is used by yarn node manager. It is difficult to judge whether to have this token or not in yarn client. The token is set into yarn application context,but the yarn client does not have API to get yarn application context.
          How do you think ? Thanks a lot!

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , I can run the yarn IT case! Thanks very much. But, in the Yarn IT case: 1、 It uses the yarn mini cluster, which is for testing.I do not know whether it is using HDFS Delegation token or not. 2、And what's more, the HDFS Delegation Token is used by yarn node manager. It is difficult to judge whether to have this token or not in yarn client. The token is set into yarn application context,but the yarn client does not have API to get yarn application context. How do you think ? Thanks a lot!
          Hide
          githubbot ASF GitHub Bot added a comment -

          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!

          Show
          githubbot ASF GitHub Bot added a comment - 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!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user StephanEwen commented on the issue:

          https://github.com/apache/flink/pull/3776

          @Rucongzhang My understanding was that the Hadoop code should automatically renew delegation tokens when a Kerberos Keytab is present. @EronWright Can you comment on that assumption?

          Show
          githubbot ASF GitHub Bot added a comment - Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3776 @Rucongzhang My understanding was that the Hadoop code should automatically renew delegation tokens when a Kerberos Keytab is present. @EronWright Can you comment on that assumption?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , yes, when configuration keytab, the hadoop code automatically renew delegation tokens .But when token and keytab are available, the hadoop use the token first, but the keytab.

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , yes, when configuration keytab, the hadoop code automatically renew delegation tokens .But when token and keytab are available, the hadoop use the token first, but the keytab.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen , we resolve this problem. We only add the HDFS delegation token in JM、TM yarn container context. And when we configuration the keytab, the JM、TM use the keytab to authentication with HDFS.

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen , we resolve this problem. We only add the HDFS delegation token in JM、TM yarn container context. And when we configuration the keytab, the JM、TM use the keytab to authentication with HDFS.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/3776

          Would like to follow up on this PR.

          @Rucongzhang can you confirm my understanding of the problem?:
          So, the root cause of the issue is that when both token AND keytab is configured, we're incorrectly using the token for authentication?

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/3776 Would like to follow up on this PR. @Rucongzhang can you confirm my understanding of the problem?: So, the root cause of the issue is that when both token AND keytab is configured, we're incorrectly using the token for authentication?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @tzulitai , you are right. There are two problems in yarn cluster mode:
          1、when we use the keytab,we do not set the HDFS delegation token to yarn container context, but yarn need.
          2、when we user keytab, and also get HDFS delegation token. The UGI contains both, but UGI use token first to communication with HDFS. The default expire time of HDFS delegation token is 7 days. Flink does not refresh the token.
          So, I resolve this problem by following solution:
          1、we user keytab and also get HDFS delegation token. The token is set to yarn container context. And the UGI only use keytab.
          Maybe the best solution I think the AM need refresh the token like spark. Maybe we can create a FILP to do this.

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @tzulitai , you are right. There are two problems in yarn cluster mode: 1、when we use the keytab,we do not set the HDFS delegation token to yarn container context, but yarn need. 2、when we user keytab, and also get HDFS delegation token. The UGI contains both, but UGI use token first to communication with HDFS. The default expire time of HDFS delegation token is 7 days. Flink does not refresh the token. So, I resolve this problem by following solution: 1、we user keytab and also get HDFS delegation token. The token is set to yarn container context. And the UGI only use keytab. Maybe the best solution I think the AM need refresh the token like spark. Maybe we can create a FILP to do this.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user EronWright commented on the issue:

          https://github.com/apache/flink/pull/3776

          @Rucongzhang thanks for the contribution. I think I understand the problem and your solution, which I will recap. I also found YARN-2704 to be useful background.

          Problem:
          1. YARN log aggregation depends on an HDFS delegation token, which it obtains from container token storage not from the UGI. In keytab mode, the Flink client doesn't upload any delegation tokens, causing log aggregation to fail.
          2. The Flink cluster doesn't renew delegation tokens. Note: Flink does renew Kerberos tickets using the keytab.
          3. When the UGI contains both a delegation token and a Kerberos ticket, the delegation token is preferred. After expiration, Flink does not fallback to using the ticket.

          Solution:
          1. Change Flink client to upload delegation tokens. Addresses problem 1.
          2 Change Flink cluster to filter out the HDFS delegation token from the tokens loaded from storage when populating the UGI. Addresses problem 3.
          3 Change JM to propagate its stored tokens to the TM, rather than the tokens from the UGI (which were filtered in (2).

          Show
          githubbot ASF GitHub Bot added a comment - Github user EronWright commented on the issue: https://github.com/apache/flink/pull/3776 @Rucongzhang thanks for the contribution. I think I understand the problem and your solution, which I will recap. I also found YARN-2704 to be useful background. Problem : 1. YARN log aggregation depends on an HDFS delegation token, which it obtains from container token storage not from the UGI. In keytab mode, the Flink client doesn't upload any delegation tokens, causing log aggregation to fail. 2. The Flink cluster doesn't renew delegation tokens. Note: Flink does renew Kerberos tickets using the keytab. 3. When the UGI contains both a delegation token and a Kerberos ticket, the delegation token is preferred. After expiration, Flink does not fallback to using the ticket. Solution : 1. Change Flink client to upload delegation tokens. Addresses problem 1. 2 Change Flink cluster to filter out the HDFS delegation token from the tokens loaded from storage when populating the UGI. Addresses problem 3. 3 Change JM to propagate its stored tokens to the TM, rather than the tokens from the UGI (which were filtered in (2).
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @EronWright ,thank you very much . yes you are right. But about solutin 1. We need only add the HDFS delegation token in yarn container context , yarn client not need refresh the token, yarn resource manager can refresh it.

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @EronWright ,thank you very much . yes you are right. But about solutin 1. We need only add the HDFS delegation token in yarn container context , yarn client not need refresh the token, yarn resource manager can refresh it.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user EronWright commented on the issue:

          https://github.com/apache/flink/pull/3776

          +1

          Show
          githubbot ASF GitHub Bot added a comment - Github user EronWright commented on the issue: https://github.com/apache/flink/pull/3776 +1
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @StephanEwen ,@tzulitai , please review the code ,if it is ok. Please help me to merge the PR. Thanks!

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @StephanEwen ,@tzulitai , please review the code ,if it is ok. Please help me to merge the PR. Thanks!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/3776

          @Rucongzhang @EronWright thanks for the explanations, the changes looks good to me then.
          I'll rebase this, perform some tests and then merge this if all goes well.

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/3776 @Rucongzhang @EronWright thanks for the explanations, the changes looks good to me then. I'll rebase this, perform some tests and then merge this if all goes well.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @tzulitai ,When I have fixed the problem, I want to write a IT case. But, in the Yarn IT case:
          1、 It uses the yarn mini cluster, which is for testing.I do not know whether it is using HDFS Delegation token or not.
          2、And what's more, the HDFS Delegation Token is used by yarn node manager. It is difficult to judge whether to have this token or not in yarn client. The token is set into yarn application context,but the yarn client does not have API to get yarn application context.
          How do you think ? Thanks a lot!

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @tzulitai ,When I have fixed the problem, I want to write a IT case. But, in the Yarn IT case: 1、 It uses the yarn mini cluster, which is for testing.I do not know whether it is using HDFS Delegation token or not. 2、And what's more, the HDFS Delegation Token is used by yarn node manager. It is difficult to judge whether to have this token or not in yarn client. The token is set into yarn application context,but the yarn client does not have API to get yarn application context. How do you think ? Thanks a lot!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user tzulitai commented on the issue:

          https://github.com/apache/flink/pull/3776

          @Rucongzhang ok, understood. I agree that in general the current `AbstractYarnClusterDescriptor` has poor separation of concerns, as is a bit hard to write contained tests. We should remember to add this perhaps when refactoring it for FLIP-6.

          I'll give this a test run on YARN and then merge it

          Show
          githubbot ASF GitHub Bot added a comment - Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/3776 @Rucongzhang ok, understood. I agree that in general the current `AbstractYarnClusterDescriptor` has poor separation of concerns, as is a bit hard to write contained tests. We should remember to add this perhaps when refactoring it for FLIP-6. I'll give this a test run on YARN and then merge it
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user Rucongzhang commented on the issue:

          https://github.com/apache/flink/pull/3776

          @tzulitai ,ok,Thanks a lot!

          Show
          githubbot ASF GitHub Bot added a comment - Github user Rucongzhang commented on the issue: https://github.com/apache/flink/pull/3776 @tzulitai ,ok,Thanks a lot!
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

          https://github.com/apache/flink/pull/3776

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/3776
          Hide
          tzulitai Tzu-Li (Gordon) Tai added a comment -

          Fixed for master via b1f3408f4cc5cca4536fe85300efcd5267eba73a
          Fixed for 1.3 via 073852b0cb891e9c4a7bfb1f834875426dedd484.

          Show
          tzulitai Tzu-Li (Gordon) Tai added a comment - Fixed for master via b1f3408f4cc5cca4536fe85300efcd5267eba73a Fixed for 1.3 via 073852b0cb891e9c4a7bfb1f834875426dedd484.

            People

            • Assignee:
              cong zhangrucong1982
              Reporter:
              cong zhangrucong1982
            • Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development