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

Greg Senia commented on YARN-5910:
----------------------------------

@jlowe @jianhe or @clayb do you know if this Jira solves the problem I have 
below where I attempt to run distcp as my user from active directory. In this 
scenerio each Hadoop Cluster has its own KDC that has a one-way trust to Active 
Directory.

In the case below I attempt to run distcp as my active directory user and it 
will not run. hadoop fs -cp hdfs://tech/public_data 
hdfs://unit/public_data/gss_test works no problem. So I am wondering if this 
Jira solves this problem of a cluster attempting to get a delegation token?

@clayb 's questions from above I guess my question is does this solve #2?

I see there are two issues here I was hoping to solve:
1. A remote cluster's services are needed (e.g. as a data source to this job)
2. A remote cluster does not trust this cluster's YARN principal

 

Distcp between clusters that are NOT allowed to trust each others kerberos 
realms:
hadoop distcp hdfs://tech/public_data hdfs://unit/public_data/gss_test

RM Log from DISTCP?
2018-02-22 11:11:45,677 INFO resourcemanager.ClientRMService 
(ClientRMService.java:submitApplication(588)) - Application with id 112 
submitted by user gss2002
2018-02-22 11:11:45,677 INFO resourcemanager.RMAuditLogger 
(RMAuditLogger.java:logSuccess(170)) - USER=gss2002 IP=10.70.40.255 
OPERATION=Submit Application Request TARGET=ClientRMService RESULT=SUCCESS 
APPID=application_1519238638021_0112 CALLERCONTEXT=CLI
2018-02-22 11:11:45,679 INFO security.DelegationTokenRenewer 
(DelegationTokenRenewer.java:handleAppSubmitEvent(449)) - 
application_1519238638021_0112 found existing hdfs token Kind: 
HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN 
token 102956 for gss2002)
2018-02-22 11:11:54,526 WARN ipc.Client (Client.java:run(717)) - Couldn't setup 
connection for rm/ha21t52mn.tech.hdp.example....@tech.hdp.example.com to 
ha21d51nn.unit.hdp.example.com/10.69.81.6:8020
javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: 
No valid credentials provided (Mechanism level: Fail to create credential. (63) 
- No service creds)]
 at 
com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:211)
 at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:414)
 at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:597)
 at org.apache.hadoop.ipc.Client$Connection.access$2100(Client.java:399)
 at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:768)
 at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:764)
 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:1740)
 at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:764)
 at org.apache.hadoop.ipc.Client$Connection.access$3300(Client.java:399)
 at org.apache.hadoop.ipc.Client.getConnection(Client.java:1626)
 at org.apache.hadoop.ipc.Client.call(Client.java:1457)
 at org.apache.hadoop.ipc.Client.call(Client.java:1404)
 at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
 at com.sun.proxy.$Proxy93.renewDelegationToken(Unknown Source)
 at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.renewDelegationToken(ClientNamenodeProtocolTranslatorPB.java:993)
 at sun.reflect.GeneratedMethodAccessor214.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:278)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:194)
 at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:176)
 at com.sun.proxy.$Proxy94.renewDelegationToken(Unknown Source)
 at org.apache.hadoop.hdfs.DFSClient$Renewer.renew(DFSClient.java:1141)
 at org.apache.hadoop.security.token.Token.renew(Token.java:414)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:597)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:594)
 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:1740)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.renewToken(DelegationTokenRenewer.java:592)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:461)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$800(DelegationTokenRenewer.java:78)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:904)
 at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:881)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
 at java.lang.Thread.run(Thread.java:745)
Caused by: GSSException: No valid credentials provided (Mechanism level: Fail 
to create credential. (63) - No service creds)
 at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:770)
 at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:248)
 at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
 at 
com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:192)
 ... 38 more
Caused by: KrbException: Fail to create credential. (63) - No service creds
 at 
sun.security.krb5.internal.CredentialsUtil.acquireServiceCreds(CredentialsUtil.java:156)
 at sun.security.krb5.Credentials.acquireServiceCreds(Credentials.java:458)
 at sun.security.jgss.krb5.Krb5Context.initSecContext(Krb5Context.java:693)
 ... 41 more

> 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.
>            Assignee: Jian He
>            Priority: Minor
>             Fix For: 2.9.0, 3.0.0-alpha4
>
>         Attachments: YARN-5910.01.patch, YARN-5910.2.patch, 
> YARN-5910.3.patch, YARN-5910.4.patch, YARN-5910.5.patch, YARN-5910.6.patch, 
> YARN-5910.7.patch
>
>
> 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
(v7.6.3#76005)

---------------------------------------------------------------------
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