[ https://issues.apache.org/jira/browse/MAPREDUCE-3529?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Siddharth Seth updated MAPREDUCE-3529: -------------------------------------- Status: Open (was: Patch Available) Cancelling - Jenkins doesn't seem to have picked up the HADOOP-7933. Thanks for taking a look Daryn, bq. If fs.getDelegationTokens(delegTokenRenewer, credentials) returns null because all tokens are already acquired, isn't this going to cause fs.getDelegationTokens(delegTokenRenewer) to be unnecessarily called? fs.getDelegationTokens(renewer, creds) is supposed to return the full list of credentials (not an incremental list). If there's no tokens - the assumption is the API may not have been implemented and hence the fallback to getDelegationToken. bq. credentials.addToken(fsNameText, token) really should be credentials.addToken(token.getService(), token). Will upload a new patch with this change. > TokenCache does not cache viewfs credentials correctly > ------------------------------------------------------ > > Key: MAPREDUCE-3529 > URL: https://issues.apache.org/jira/browse/MAPREDUCE-3529 > Project: Hadoop Map/Reduce > Issue Type: Bug > Components: mrv2 > Affects Versions: 0.23.0 > Reporter: Siddharth Seth > Assignee: Siddharth Seth > Priority: Critical > Attachments: MR3529_v1.txt, MR3529_v2.txt > > > viewfs returns a list of delegation tokens for the actual namenodes. > TokenCache caches these based on the actual service name - subsequent calls > to TokenCache end up trying to get a new set of tokens. > Tasks which happen to access TokenCache fail when using viewfs - since they > end up trying to get a new set of tokens even though the tokens are already > available. > {noformat} > Error: java.io.IOException: Delegation Token can be issued only with kerberos > or web authentication > at > org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getDelegationToken(FSNamesystem.java:4027) > at > org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getDelegationToken(NameNodeRpcServer.java:281) > at sun.reflect.GeneratedMethodAccessor38.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at > org.apache.hadoop.ipc.WritableRpcEngine$Server.call(WritableRpcEngine.java:365) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1490) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:1486) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:396) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1152) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:1484) > at org.apache.hadoop.ipc.Client.call(Client.java:1085) > at > org.apache.hadoop.ipc.WritableRpcEngine$Invoker.invoke(WritableRpcEngine.java:193) > at $Proxy8.getDelegationToken(Unknown Source) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) > at java.lang.reflect.Method.invoke(Method.java:597) > at > org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:100) > at > org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:65) > at $Proxy8.getDelegationToken(Unknown Source) > at > org.apache.hadoop.hdfs.DFSClient.getDelegationToken(DFSClient.java:456) > at > org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationToken(DistributedFileSystem.java:812) > at > org.apache.hadoop.hdfs.DistributedFileSystem.getDelegationTokens(DistributedFileSystem.java:839) > at > org.apache.hadoop.fs.viewfs.ChRootedFileSystem.getDelegationTokens(ChRootedFileSystem.java:311) > at > org.apache.hadoop.fs.viewfs.ViewFileSystem.getDelegationTokens(ViewFileSystem.java:490) > at > org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:144) > at > org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:91) > at > org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes(TokenCache.java:84) > {noformat} > This will likely require some changes in viewfs/hdfs - will open a Jira with > details. -- This message is automatically generated by JIRA. If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa For more information on JIRA, see: http://www.atlassian.com/software/jira