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

Jian He commented on YARN-5910:
-------------------------------

bq. ideally, the token should be self-sufficient to discover the renewer 
address.
After digging the code more for this approach, even in non-HA mode, conf is 
also required for things like retry settings, also the principal name is 
required for secure setting.  Basically the Token has to selectively carry all 
the necessary conf for connecting to the renewer in HA, non-HA, secure 
scenarios.  How to maintain such an unknown list is a non-trivial task in the 
first place. I'd prefer the passing via appSubmissionContext approach now. 

> Support for multi-cluster delegation tokens
> -------------------------------------------
>
>                 Key: YARN-5910
>                 URL: https://issues.apache.org/jira/browse/YARN-5910
>             Project: Hadoop YARN
>          Issue Type: New Feature
>          Components: security
>            Reporter: Clay B.
>            Priority: Minor
>
> As an administrator running many secure (kerberized) clusters, some which 
> have peer clusters managed by other teams, I am looking for a way to run jobs 
> which may require services running on other clusters. Particular cases where 
> this rears itself are running something as core as a distcp between two 
> kerberized clusters (e.g. {{hadoop --config /home/user292/conf/ distcp 
> hdfs://LOCALCLUSTER/user/user292/test.out 
> hdfs://REMOTECLUSTER/user/user292/test.out.result}}).
> Thanks to YARN-3021, once can run for a while but if the delegation token for 
> the remote cluster needs renewal the job will fail[1]. One can pre-configure 
> their {{hdfs-site.xml}} loaded by the YARN RM to know of all possible HDFSes 
> available but that requires coordination that is not always feasible, 
> especially as a cluster's peers grow into the tens of clusters or across 
> management teams. Ideally, one could have core systems configured this way 
> but jobs could also specify their own handling of tokens and management when 
> needed?
> [1]: Example stack trace when the RM is unaware of a remote service:
> ----------------
> {code}
> 2016-03-23 14:59:50,528 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer:
>  application_1458441356031_3317 found existing hdfs token Kind: 
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:REMOTECLUSTER, Ident: 
> (HDFS_DELEGATION_TOKEN token
>  10927 for user292)
> 2016-03-23 14:59:50,557 WARN 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer:
>  Unable to add the application to the delegation token renewer.
> java.io.IOException: Failed to renew token: Kind: HDFS_DELEGATION_TOKEN, 
> Service: ha-hdfs:REMOTECLUSTER, Ident: (HDFS_DELEGATION_TOKEN token 10927 for 
> user292)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:427)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$700(DelegationTokenRenewer.java:78)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:781)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:762)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
> Caused by: java.io.IOException: Unable to map logical nameservice URI 
> 'hdfs://REMOTECLUSTER' to a NameNode. Local configuration does not have a 
> failover proxy provider configured.
> at org.apache.hadoop.hdfs.DFSClient$Renewer.getNNProxy(DFSClient.java:1164)
> at org.apache.hadoop.hdfs.DFSClient$Renewer.renew(DFSClient.java:1128)
> at org.apache.hadoop.security.token.Token.renew(Token.java:377)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:516)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:513)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.renewToken(DelegationTokenRenewer.java:511)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:425)
> ... 6 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org

Reply via email to