hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Hadoop QA (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode
Date Fri, 23 Oct 2015 08:24:27 GMT

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

Hadoop QA commented on HDFS-9276:
---------------------------------

\\
\\
| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | pre-patch |  19m 22s | Pre-patch trunk compilation is healthy. |
| {color:green}+1{color} | @author |   0m  0s | The patch does not contain any @author tags.
|
| {color:red}-1{color} | tests included |   0m  0s | The patch doesn't appear to include any
new or modified tests.  Please justify why no new tests are needed for this patch. Also please
list what manual steps were performed to verify this patch. |
| {color:green}+1{color} | javac |   9m 11s | There were no new javac warning messages. |
| {color:green}+1{color} | javadoc |  12m 15s | There were no new javadoc warning messages.
|
| {color:green}+1{color} | release audit |   0m 24s | The applied patch does not increase
the total number of release audit warnings. |
| {color:red}-1{color} | checkstyle |   1m 15s | The applied patch generated  1 new checkstyle
issues (total was 28, now 29). |
| {color:green}+1{color} | whitespace |   0m  0s | The patch has no lines that end in whitespace.
|
| {color:green}+1{color} | install |   1m 46s | mvn install still works. |
| {color:green}+1{color} | eclipse:eclipse |   0m 37s | The patch built with eclipse:eclipse.
|
| {color:red}-1{color} | findbugs |   2m  9s | The patch appears to introduce 1 new Findbugs
(version 3.0.0) warnings. |
| {color:red}-1{color} | common tests |   8m  2s | Tests failed in hadoop-common. |
| | |  55m  6s | |
\\
\\
|| Reason || Tests ||
| FindBugs | module:hadoop-common |
| Failed unit tests | hadoop.ipc.TestDecayRpcScheduler |
\\
\\
|| Subsystem || Report/Notes ||
| Patch URL | http://issues.apache.org/jira/secure/attachment/12768233/HDFS-9276.01.patch
|
| Optional Tests | javadoc javac unit findbugs checkstyle |
| git revision | trunk / 124a412 |
| checkstyle |  https://builds.apache.org/job/PreCommit-HDFS-Build/13148/artifact/patchprocess/diffcheckstylehadoop-common.txt
|
| Findbugs warnings | https://builds.apache.org/job/PreCommit-HDFS-Build/13148/artifact/patchprocess/newPatchFindbugsWarningshadoop-common.html
|
| hadoop-common test log | https://builds.apache.org/job/PreCommit-HDFS-Build/13148/artifact/patchprocess/testrun_hadoop-common.txt
|
| Test Results | https://builds.apache.org/job/PreCommit-HDFS-Build/13148/testReport/ |
| Java | 1.7.0_55 |
| uname | Linux asf909.gq1.ygridcore.net 3.13.0-36-lowlatency #63-Ubuntu SMP PREEMPT Wed Sep
3 21:56:12 UTC 2014 x86_64 x86_64 x86_64 GNU/Linux |
| Console output | https://builds.apache.org/job/PreCommit-HDFS-Build/13148/console |


This message was automatically generated.

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