[
https://issues.apache.org/jira/browse/SAMZA-929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16499726#comment-16499726
]
Apoorva Sareen commented on SAMZA-929:
--------------------------------------
Hi [~nickpan47],
We have seen an instance of this issue which we can consistently reproduce in
our environment. We are using Hadoop 2.7.4 locally and running samza 0.11.0
version.
We have turned on *yarn.resourcemanager.proxy-user-privileges.enabled* and
added yarn as proxy user to core-site.xml
If i kill the application master process, yarn restarts it again successfully
even when original launcher token has expired. Log aggregation also works fine
in our environment. However, we face issues in launch of worker container
(after AM restart) which repeatedly fail with following error
2018-06-04-03:36:31.916 [Container Allocator Thread] ERROR
org.apache.samza.job.yarn.YarnContainerRunner - IO Exception when accessing the
package status from the filesystem org.apache.hadoop.ipc.RemoteException: token
*(HDFS_DELEGATION_TOKEN token 26412 for centos)* can't be found in cache at
org.apache.hadoop.ipc.Client.call(Client.java:1470)
~[hadoop-common-2.6.5.jar:?] at
org.apache.hadoop.ipc.Client.call(Client.java:1401)
~[hadoop-common-2.6.5.jar:?] at
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
~[hadoop-common-2.6.5.jar:?] at com.sun.proxy.$Proxy34.getFileInfo(Unknown
Source) ~[?:?] at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:752)
~[hadoop-hdfs-2.6.5.jar:?] at
sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_131] at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
~[?:1.8.0_131] at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
~[?:1.8.0_131] at java.lang.reflect.Method.invoke(Method.java:498)
~[?:1.8.0_131] at
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
~[hadoop-common-2.6.5.jar:?] at
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
~[hadoop-common-2.6.5.jar:?] at com.sun.proxy.$Proxy35.getFileInfo(Unknown
Source) ~[?:?] at
org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1977)
~[hadoop-hdfs-2.6.5.jar:?] at
org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1118)
~[hadoop-hdfs-2.6.5.jar:?] at
org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1114)
~[hadoop-hdfs-2.6.5.jar:?] at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
~[hadoop-common-2.6.5.jar:?] at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1114)
~[hadoop-hdfs-2.6.5.jar:?] at
org.apache.samza.job.yarn.YarnContainerRunner.startContainer(YarnContainerRunner.java:159)
[samza-yarn_2.11-0.11.0.jar:?] at
org.apache.samza.job.yarn.YarnContainerRunner.runContainer(YarnContainerRunner.java:118)
[samza-yarn_2.11-0.11.0.jar:?] at
org.apache.samza.job.yarn.YarnClusterResourceManager.launchStreamProcessor(YarnClusterResourceManager.java:259)
[samza-yarn_2.11-0.11.0.jar:?] at
org.apache.samza.clustermanager.AbstractContainerAllocator.runStreamProcessor(AbstractContainerAllocator.java:156)
[samza-core_2.11-0.11.0.jar:0.11.0] at
org.apache.samza.clustermanager.ContainerAllocator.assignResourceRequests(ContainerAllocator.java:52)
[samza-core_2.11-0.11.0.jar:0.11.0] at
org.apache.samza.clustermanager.AbstractContainerAllocator.run(AbstractContainerAllocator.java:109)
[samza-core_2.11-0.11.0.jar:0.11.0] at java.lang.Thread.run(Thread.java:748)
[?:1.8.0_131]
HDFS_DELEGATION_TOKEN token 26412 here is the original token which was set in
the launcer context. This token has already expired. Though, after
yarn.token.renewal.interval.seconds, *SamzaAppMasterSecurityManager* re-logins
using key tab and generates a new TOKEN. There after, the containers get
launched successfully with this new Token. But this could cause significant
delay and data loss
I noticed that in *SamzaAppMasterSecurityManager* there is ** following line
which creates the tokenRenewal task
tokenRenewExecutor.scheduleAtFixedRate(tokenRenewRunnable, +*renewalInterval*+,
renewalInterval, TimeUnit.SECONDS)
Does is make sense to re-login to keytab from app master as soon as it starts
by setting the underline parameter (that represents initialDelay) to zero (or a
small value)?
Regards,
Apoorva
> Support AM restart in Kerberos secured YARN after the original token in
> launch context expires
> ----------------------------------------------------------------------------------------------
>
> Key: SAMZA-929
> URL: https://issues.apache.org/jira/browse/SAMZA-929
> Project: Samza
> Issue Type: Sub-task
> Components: yarn
> Reporter: Yi Pan (Data Infrastructure)
> Assignee: Chen Song
> Priority: Major
>
> In Kerberos secured YARN cluster, an initial token is given in the launch
> context to RM for a job. When that token expires, RM will not be able to
> restart failed AM. As for now, we will have to re-submit the job to refresh
> the token in the launch context.
> This ticket is searching for a solution that can refresh the token in the
> launch context for AM.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)