You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Liangliang Gu (JIRA)" <ji...@apache.org> on 2015/10/21 11:03:27 UTC

[jira] [Created] (HADOOP-12497) Failed to Update HDFS Delegation Token for long running application in HA mode

Liangliang Gu created HADOOP-12497:
--------------------------------------

             Summary: Failed to Update HDFS Delegation Token for long running application in HA mode
                 Key: HADOOP-12497
                 URL: https://issues.apache.org/jira/browse/HADOOP-12497
             Project: Hadoop Common
          Issue Type: Bug
          Components: fs, ha
            Reporter: Liangliang Gu


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:
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
      }
    })
  }
}

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.

The stacktrace is:
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)



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