hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yi Liu (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode
Date Mon, 26 Oct 2015 05:40:27 GMT

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

Yi Liu edited comment on HDFS-9276 at 10/26/15 5:39 AM:
--------------------------------------------------------

{quote}
To reproduce the bug, please set the following configuration to Name Node:
dfs.namenode.delegation.token.max-lifetime = 10min
dfs.namenode.delegation.key.update-interval = 3min
dfs.namenode.delegation.token.renew-interval = 3min
The bug will occure after 3 minutes.
{quote}

Your test code can't say anything,  the error msg of "token (HDFS_DELEGATION_TOKEN token 330156
for test) is expired" is because you set "dfs.namenode.delegation.token.renew-interval" to
3 min but you don't let {{test}} user to renew the token. 

I see what you want to do now, if the same with the later case as I commented above.  Actually
hadoop code is enough to let you do what you want to do.  If a user client get a new delegation
token, and your long running application can accept it, you can update the credentials of
user's UGI on the server through {{UserGroupInformation#addCredentials}}, it will overwrite
old tokens by default, of course you should make the service name of token is the same if
you want to overwrite it.

It's not a bug.


was (Author: hitliuyi):
{quote}
To reproduce the bug, please set the following configuration to Name Node:
dfs.namenode.delegation.token.max-lifetime = 10min
dfs.namenode.delegation.key.update-interval = 3min
dfs.namenode.delegation.token.renew-interval = 3min
The bug will occure after 3 minutes.
{quote}

Your test code can't say anything,  the error msg of "token (HDFS_DELEGATION_TOKEN token 330156
for test) is expired" is because you set "dfs.namenode.delegation.token.renew-interval" to
3 min but you don't let {{test}} user to renew the token. 

I see what you want to do now.  Actually hadoop code is enough to let you do what you want
to do.  If a user client get a new delegation token, and your long running application can
accept it, you can update the credentials of user's UGI on the server through {{UserGroupInformation#addCredentials}},
it will overwrite old tokens by default, of course you should make the service name of token
is the same if you want to overwrite it.

It's not a bug.

> Failed to Update HDFS Delegation Token for long running application in HA mode
> ------------------------------------------------------------------------------
>
>                 Key: HDFS-9276
>                 URL: https://issues.apache.org/jira/browse/HDFS-9276
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: fs, ha, security
>    Affects Versions: 2.7.1
>            Reporter: Liangliang Gu
>            Assignee: Liangliang Gu
>         Attachments: HDFS-9276.01.patch, HDFS-9276.02.patch, HDFS-9276.03.patch, debug1.PNG,
debug2.PNG
>
>
> The Scenario is as follows:
> 1. NameNode HA is enabled.
> 2. Kerberos is enabled.
> 3. HDFS Delegation Token (not Keytab or TGT) is used to communicate with NameNode.
> 4. We want to update the HDFS Delegation Token for long running applicatons. HDFS Client
will generate private tokens for each NameNode. When we update the HDFS Delegation Token,
these private tokens will not be updated, which will cause token expired.
> This bug can be reproduced by the following program:
> {code}
> import java.security.PrivilegedExceptionAction
> import org.apache.hadoop.conf.Configuration
> import org.apache.hadoop.fs.{FileSystem, Path}
> import org.apache.hadoop.security.UserGroupInformation
> object HadoopKerberosTest {
>   def main(args: Array[String]): Unit = {
>     val keytab = "/path/to/keytab/xxx.keytab"
>     val principal = "xxx@ABC.COM"
>     val creds1 = new org.apache.hadoop.security.Credentials()
>     val ugi1 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
>     ugi1.doAs(new PrivilegedExceptionAction[Void] {
>       // Get a copy of the credentials
>       override def run(): Void = {
>         val fs = FileSystem.get(new Configuration())
>         fs.addDelegationTokens("test", creds1)
>         null
>       }
>     })
>     val ugi = UserGroupInformation.createRemoteUser("test")
>     ugi.addCredentials(creds1)
>     ugi.doAs(new PrivilegedExceptionAction[Void] {
>       // Get a copy of the credentials
>       override def run(): Void = {
>         var i = 0
>         while (true) {
>           val creds1 = new org.apache.hadoop.security.Credentials()
>           val ugi1 = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal,
keytab)
>           ugi1.doAs(new PrivilegedExceptionAction[Void] {
>             // Get a copy of the credentials
>             override def run(): Void = {
>               val fs = FileSystem.get(new Configuration())
>               fs.addDelegationTokens("test", creds1)
>               null
>             }
>           })
>           UserGroupInformation.getCurrentUser.addCredentials(creds1)
>           val fs = FileSystem.get( new Configuration())
>           i += 1
>           println()
>           println(i)
>           println(fs.listFiles(new Path("/user"), false))
>           Thread.sleep(60 * 1000)
>         }
>         null
>       }
>     })
>   }
> }
> {code}
> To reproduce the bug, please set the following configuration to Name Node:
> {code}
> dfs.namenode.delegation.token.max-lifetime = 10min
> dfs.namenode.delegation.key.update-interval = 3min
> dfs.namenode.delegation.token.renew-interval = 3min
> {code}
> The bug will occure after 3 minutes.
> The stacktrace is:
> {code}
> Exception in thread "main" org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken):
token (HDFS_DELEGATION_TOKEN token 330156 for test) is expired
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1347)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1300)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
> 	at com.sun.proxy.$Proxy9.getFileInfo(Unknown Source)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:651)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:606)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> 	at com.sun.proxy.$Proxy10.getFileInfo(Unknown Source)
> 	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1679)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1106)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1102)
> 	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1102)
> 	at org.apache.hadoop.fs.FileSystem.resolvePath(FileSystem.java:747)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem$15.<init>(DistributedFileSystem.java:726)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.listLocatedStatus(DistributedFileSystem.java:717)
> 	at org.apache.hadoop.fs.FileSystem.listLocatedStatus(FileSystem.java:1780)
> 	at org.apache.hadoop.fs.FileSystem$5.<init>(FileSystem.java:1842)
> 	at org.apache.hadoop.fs.FileSystem.listFiles(FileSystem.java:1839)
> 	at HadoopKerberosTest6$$anon$2.run(HadoopKerberosTest6.scala:55)
> 	at HadoopKerberosTest6$$anon$2.run(HadoopKerberosTest6.scala:32)
> 	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:1491)
> 	at HadoopKerberosTest6$.main(HadoopKerberosTest6.scala:32)
> 	at HadoopKerberosTest6.main(HadoopKerberosTest6.scala)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:606)
> 	at com.intellij.rt.execution.application.AppMain.main(AppMain.java:140)
> {code}



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

Mime
View raw message