[jira] [Comment Edited] (FLINK-33155) Flink ResourceManager continuously fails to start TM container on YARN when Kerberos enabled

2023-09-30 Thread Jira


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

袁枫 edited comment on FLINK-33155 at 9/30/23 7:25 AM:
-

The root cause might be that we are reading the delegation token from JM local 
file[1] , as i know token will renew every 1 day by Yarn rm until reach 7days 
limit, token in your log maxDate is ‘2023-09-27 15:53:51.487’, error ocurr is 
'2023-9-25', it is say: althought jm use token in local file, it is also can be 
sucesseful. If the token be renewed in 2023-9-25 sometime by yarn rm. So i 
think if this is a yarn issue? [~wangyang0918], hope your reply! 


was (Author: feng yuan):
The root cause might be that we are reading the delegation token from JM local 
file[1] , as i know token will renew every 1 day by Yarn rm until reach 7days 
limit, token in your log maxDate is ‘2023-09-27 15:53:51.487’, error ocurr is 
'2023-9-25', it is say: althought jm use token in local file, it is also can be 
sucesseful. If the token be renewed in 2023-9-25 sometime by yarn rm. So i 
think if this is a yarn issue? [~wangyang0918], help your reply! 

> Flink ResourceManager continuously fails to start TM container on YARN when 
> Kerberos enabled
> 
>
> Key: FLINK-33155
> URL: https://issues.apache.org/jira/browse/FLINK-33155
> Project: Flink
>  Issue Type: Bug
>  Components: Deployment / YARN
>Reporter: Yang Wang
>Priority: Major
>
> When Kerberos enabled(with key tab) and after one day(the container token 
> expired), Flink fails to create the TaskManager container on YARN due to the 
> following exception.
>  
> {code:java}
> 2023-09-25 16:48:50,030 INFO  
> org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager [] - 
> Worker container_1695106898104_0003_01_69 is terminated. Diagnostics: 
> Container container_1695106898104_0003_01_69 was invalid. Diagnostics: 
> [2023-09-25 16:48:45.710]token (token for hadoop: HDFS_DELEGATION_TOKEN 
> owner=hadoop/master-1-1.c-5ee7bdc598b6e1cc.cn-beijing.emr.aliyuncs@emr.c-5ee7bdc598b6e1cc.com,
>  renewer=, realUser=, issueDate=1695196431487, maxDate=1695801231487, 
> sequenceNumber=12, masterKeyId=3) can't be found in cache
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
>  token (token for hadoop: HDFS_DELEGATION_TOKEN owner=, renewer=, 
> realUser=, issueDate=1695196431487, maxDate=1695801231487, sequenceNumber=12, 
> masterKeyId=3) can't be found in cache
>     at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1545)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1491)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1388)
>     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.getFileInfo(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:907)
>     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:431)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:166)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:158)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:96)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:362)
>     at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
>     at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1666)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1576)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1573)
>     at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1588)
>     at 
> org.apache.hadoop.yarn.util.FSDownload.verifyAndCopy(FSDownload.java:269)
>     at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:67)
>     at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:414)
>     at 

[jira] [Comment Edited] (FLINK-33155) Flink ResourceManager continuously fails to start TM container on YARN when Kerberos enabled

2023-09-29 Thread Jira


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

袁枫 edited comment on FLINK-33155 at 9/30/23 4:22 AM:
-

The root cause might be that we are reading the delegation token from JM local 
file[1] , as i know token will renew every 1 day by Yarn rm until reach 7days 
limit, token in your log maxDate is ‘2023-09-27 15:53:51.487’, error ocurr is 
'2023-9-25', it is say: althought jm use token in local file, it is also can be 
sucesseful. If the token be renewed in 2023-9-25 sometime by yarn rm. So i 
think if this is a yarn issue? [~wangyang0918], help your reply! 


was (Author: feng yuan):
The root cause might be that we are reading the delegation token from JM local 
file[1] , as i know token will renew every 1 day by Yarn rm, token in your log 
maxDate is ‘2023-09-27 15:53:51.487’, error ocurr is '2023-9-25', it is say: 
althought jm use token in local file, it is also can be sucesseful. If the 
token be renewed in 2023-9-25 sometime by yarn rm. So i think if this is a yarn 
issue? [~wangyang0918], help your reply! 

> Flink ResourceManager continuously fails to start TM container on YARN when 
> Kerberos enabled
> 
>
> Key: FLINK-33155
> URL: https://issues.apache.org/jira/browse/FLINK-33155
> Project: Flink
>  Issue Type: Bug
>  Components: Deployment / YARN
>Reporter: Yang Wang
>Priority: Major
>
> When Kerberos enabled(with key tab) and after one day(the container token 
> expired), Flink fails to create the TaskManager container on YARN due to the 
> following exception.
>  
> {code:java}
> 2023-09-25 16:48:50,030 INFO  
> org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager [] - 
> Worker container_1695106898104_0003_01_69 is terminated. Diagnostics: 
> Container container_1695106898104_0003_01_69 was invalid. Diagnostics: 
> [2023-09-25 16:48:45.710]token (token for hadoop: HDFS_DELEGATION_TOKEN 
> owner=hadoop/master-1-1.c-5ee7bdc598b6e1cc.cn-beijing.emr.aliyuncs@emr.c-5ee7bdc598b6e1cc.com,
>  renewer=, realUser=, issueDate=1695196431487, maxDate=1695801231487, 
> sequenceNumber=12, masterKeyId=3) can't be found in cache
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
>  token (token for hadoop: HDFS_DELEGATION_TOKEN owner=, renewer=, 
> realUser=, issueDate=1695196431487, maxDate=1695801231487, sequenceNumber=12, 
> masterKeyId=3) can't be found in cache
>     at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1545)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1491)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1388)
>     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.getFileInfo(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:907)
>     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:431)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:166)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:158)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:96)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:362)
>     at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
>     at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1666)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1576)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1573)
>     at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1588)
>     at 
> org.apache.hadoop.yarn.util.FSDownload.verifyAndCopy(FSDownload.java:269)
>     at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:67)
>     at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:414)
>     at 

[jira] [Comment Edited] (FLINK-33155) Flink ResourceManager continuously fails to start TM container on YARN when Kerberos enabled

2023-09-25 Thread Gabor Somogyi (Jira)


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

Gabor Somogyi edited comment on FLINK-33155 at 9/25/23 12:20 PM:
-

Not updating UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION is a known 
limitation of YARN.

If the mentioned code runs on the JM side and delegation tokens are enabled 
then it makes sense since the JM keeps it's tokens up-to-date all the time.

Couple of notes:
 * Changing the default behavior from file to UGI can be a breaking change to 
users which are depending on that some way
 * DT handling is a single threaded operation but as I see TM creation uses 
multiple threads which may end-up in undefined behavior


was (Author: gaborgsomogyi):
Not updating UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION is a known 
limitation of YARN.

If the mentioned code runs on the JM side and delegation tokens are enabled 
then it makes sense since the JM keeps it's tokens up-to-date all the time.

Couple of notes:
 * Changing the default behavior from file to UGI can be a breaking change to 
users which are depending on that some way...

> Flink ResourceManager continuously fails to start TM container on YARN when 
> Kerberos enabled
> 
>
> Key: FLINK-33155
> URL: https://issues.apache.org/jira/browse/FLINK-33155
> Project: Flink
>  Issue Type: Bug
>  Components: Deployment / YARN
>Reporter: Yang Wang
>Priority: Major
>
> When Kerberos enabled(with key tab) and after one day(the container token 
> expired), Flink fails to create the TaskManager container on YARN due to the 
> following exception.
>  
> {code:java}
> 2023-09-25 16:48:50,030 INFO  
> org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager [] - 
> Worker container_1695106898104_0003_01_69 is terminated. Diagnostics: 
> Container container_1695106898104_0003_01_69 was invalid. Diagnostics: 
> [2023-09-25 16:48:45.710]token (token for hadoop: HDFS_DELEGATION_TOKEN 
> owner=hadoop/master-1-1.c-5ee7bdc598b6e1cc.cn-beijing.emr.aliyuncs@emr.c-5ee7bdc598b6e1cc.com,
>  renewer=, realUser=, issueDate=1695196431487, maxDate=1695801231487, 
> sequenceNumber=12, masterKeyId=3) can't be found in cache
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
>  token (token for hadoop: HDFS_DELEGATION_TOKEN owner=, renewer=, 
> realUser=, issueDate=1695196431487, maxDate=1695801231487, sequenceNumber=12, 
> masterKeyId=3) can't be found in cache
>     at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1545)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1491)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1388)
>     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.getFileInfo(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:907)
>     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:431)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:166)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:158)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:96)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:362)
>     at com.sun.proxy.$Proxy11.getFileInfo(Unknown Source)
>     at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1666)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1576)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1573)
>     at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>     at 
> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1588)
>     at 
> org.apache.hadoop.yarn.util.FSDownload.verifyAndCopy(FSDownload.java:269)
>     at org.apache.hadoop.yarn.util.FSDownload.access$000(FSDownload.java:67)
>     at org.apache.hadoop.yarn.util.FSDownload$2.run(FSDownload.java:414)
>     at