[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2016-07-24 Thread John Zhuge (JIRA)

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

John Zhuge edited comment on HDFS-9276 at 7/24/16 11:13 PM:


Another thing that bothers me is {{Token}} copy constructor performs shallow 
copy of all 4 fields. Is this by design? Could [~jnp] or [~owen.omalley] please 
comment since it is added by MAPREDUCE-2764?
{code}
 public Token(Token other) {
this.identifier = other.identifier;   <<< byte[]
this.password = other.password;   <<< byte[]
this.kind = other.kind;   <<< Text
this.service = other.service; <<< Text
  }
{code}


was (Author: jzhuge):
Another thing that bothers me is {{Token}} copy constructor performs shallow 
copy of all 4 fields. Is this by design?
{code}
 public Token(Token other) {
this.identifier = other.identifier;
this.password = other.password;
this.kind = other.kind;
this.service = other.service;
  }
{code}

> 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, HDFS-9276.04.patch, HDFS-9276.05.patch, 
> HDFS-9276.06.patch, HDFS-9276.07.patch, HDFS-9276.08.patch, 
> HDFS-9276.09.patch, HDFS-9276.10.patch, HDFS-9276.11.patch, 
> HDFS-9276.12.patch, HDFS-9276.13.patch, HDFS-9276.14.patch, 
> HDFS-9276.15.patch, HDFS-9276.16.patch, HDFSReadLoop.scala, 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 = "x...@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 

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2016-07-23 Thread John Zhuge (JIRA)

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

John Zhuge edited comment on HDFS-9276 at 7/23/16 4:36 PM:
---

Patch 16:
* Fix test {{TestUserGroupInformation}} failure
* Fix javac unchecked warning by replacing {{new Token.PrivateToken(t)}} with 
{{new Token.PrivateToken<>(t)}}. No need to suppress the warning.
{noformat}
[unchecked] unchecked call to PrivateToken(Token) as a member of the raw 
type PrivateToken
{noformat}
* Pass these unit tests: 
{{TestUserGroupInformation,TestDelegationTokenForProxyUser,TestDataNodeLifeline,TestCrcCorruption,TestBlockTokenWithDFSStriped,TestHttpServerLifecycle,TestLeaseRecovery2}}


was (Author: jzhuge):
Patch 16:
* Fix test {{TestUserGroupInformation}} failure
* Fix javac unchecked warning without suppressing it:
{noformat}
[unchecked] unchecked call to PrivateToken(Token) as a member of the raw 
type PrivateToken
{noformat}
* Pass these unit tests: 
{{TestUserGroupInformation,TestDelegationTokenForProxyUser,TestDataNodeLifeline,TestCrcCorruption,TestBlockTokenWithDFSStriped,TestHttpServerLifecycle,TestLeaseRecovery2}}

> 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, HDFS-9276.04.patch, HDFS-9276.05.patch, 
> HDFS-9276.06.patch, HDFS-9276.07.patch, HDFS-9276.08.patch, 
> HDFS-9276.09.patch, HDFS-9276.10.patch, HDFS-9276.11.patch, 
> HDFS-9276.12.patch, HDFS-9276.13.patch, HDFS-9276.14.patch, 
> HDFS-9276.15.patch, HDFS-9276.16.patch, HDFSReadLoop.scala, 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 = "x...@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 

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2016-07-21 Thread John Zhuge (JIRA)

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

John Zhuge edited comment on HDFS-9276 at 7/22/16 3:57 AM:
---

I was able to reproduce with a simple long running Spark Scala application 
{{HDFSReadLoop.scala}} (attached) that performs HDFS read periodically. It hits 
the delegation toke expired exception after a little over 3 minutes. The 
cluster is HA with custom delegation properties specified in Description. 
{{HDFS-9276.13.patch}} does fix the issue.


was (Author: jzhuge):
I was able to reproduce with a simple long running Spark application 
{{HDFSReadLoop}} that performs HDFS read periodically. It hits the delegation 
toke expired exception after a little over 3 minutes. The cluster is HA with 
custom delegation properties specified in Description. {{HDFS-9276.13.patch}} 
does fix the issue.

> 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, HDFS-9276.04.patch, HDFS-9276.05.patch, 
> HDFS-9276.06.patch, HDFS-9276.07.patch, HDFS-9276.08.patch, 
> HDFS-9276.09.patch, HDFS-9276.10.patch, HDFS-9276.11.patch, 
> HDFS-9276.12.patch, HDFS-9276.13.patch, HDFSReadLoop.scala, 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 = "x...@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)

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2016-07-21 Thread John Zhuge (JIRA)

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

John Zhuge edited comment on HDFS-9276 at 7/22/16 3:14 AM:
---

I was able to reproduce with a simple long running Spark application 
{{HDFSReadLoop}} that performs HDFS read periodically. It hits the delegation 
toke expired exception after a little over 3 minutes. The cluster is HA with 
custom delegation properties specified in Description. {{HDFS-9276.13.patch}} 
does fix the issue.


was (Author: jzhuge):
I was able to reproduce with a simple long running Spark application 
{{HDFSReadLoop}} that performs HDFS read periodically. It hits the delegation 
toke expired exception after a little over 3 minutes. {{HDFS-9276.13.patch}} 
does fix the issue.

> 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, HDFS-9276.04.patch, HDFS-9276.05.patch, 
> HDFS-9276.06.patch, HDFS-9276.07.patch, HDFS-9276.08.patch, 
> HDFS-9276.09.patch, HDFS-9276.10.patch, HDFS-9276.11.patch, 
> HDFS-9276.12.patch, HDFS-9276.13.patch, HDFSReadLoop.scala, 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 = "x...@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 
> 

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2015-10-27 Thread Yi Liu (JIRA)

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

Yi Liu edited comment on HDFS-9276 at 10/27/15 2:31 PM:


Now looks good overall.
*1.*
{code}
+
+  /**
+   * Create a private token for HA failover proxy.
+   * @return the private token
+   */
+  public PrivateToken createPrivateToken() {
+return new PrivateToken(this);
+  }
{code}
Now, we can remove this.

*2.*
{code}
-} else if (right == null || getClass() != right.getClass()) {
+} else if (right == null || !(right instanceof Token)) {
{code}
Any reason to change this? It violates {{equals}} definition.

*3.*
We should have test for real case, maybe like following:
- build a MiniDFSCluster with HA and security enabled.
- Gets delegation token
- Use the delegation token to access HDFS.
- Invalid the delegation token in NN. Check the access failed.
- Get a new delegation token and update through user's UGI.
- Check the access again, should be successful.

You can refer to existing tests about how to enable HA and security.


was (Author: hitliuyi):
Now looks good overall.
*1.*
{code}
+
+  /**
+   * Create a private token for HA failover proxy.
+   * @return the private token
+   */
+  public PrivateToken createPrivateToken() {
+return new PrivateToken(this);
+  }
{code}
Now, we can remove this.

*2.*
{code}
-} else if (right == null || getClass() != right.getClass()) {
+} else if (right == null || !(right instanceof Token)) {
{code}
Any reason to change this? It violates {{equals}} definition.

*3.*
We should have test for real case, maybe like following:
- build a MiniDFSCluster with HA and security enabled.
- Gets delegation token
- Use the delegation token to access HDFS.
- Invalid the delegation token in NN. Check the access failed.
- Get a new delegation token and update through user's UGI.
- Check the access again, should be successful.

> 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, HDFS-9276.04.patch, HDFS-9276.05.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 = "x...@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)
> }
> 

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2015-10-25 Thread Yi Liu (JIRA)

[ 
https://issues.apache.org/jira/browse/HDFS-9276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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 = "x...@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
> }
>   })
>   

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2015-10-25 Thread Yi Liu (JIRA)

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

Yi Liu edited comment on HDFS-9276 at 10/26/15 5:41 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, it's the same with the later case of what 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, 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.

> 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 = "x...@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
>

[jira] [Comment Edited] (HDFS-9276) Failed to Update HDFS Delegation Token for long running application in HA mode

2015-10-25 Thread Yi Liu (JIRA)

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

Yi Liu edited comment on HDFS-9276 at 10/25/15 1:57 PM:


[~marsishandsome],  Agree with Steve you need to move this to Hadoop common if 
the patch only contains change in common.  Before this, I think you may have a 
mistake about how to use delegation token.

Do you want to update the delegation token through 
{{FileSystem#addDelegationTokens}}?  It will not get new delegation token again 
if old one exists in the credentials, also it may be more complicate than what 
you think.
Actually I am curious about how you write your long running application. Is 
your application just an user client or a separate service?  If your 
application is just one user client, I mean it's not a service which is 
accessed by many user clients, then you still need to user Kerberos instead of 
delegationToken, but if your application is a real service which serves user 
clients, then the delegation token is the right one. The delegation token is 
used in your service to access HDFS on behalf the user, usually your 
application service can renew the delegation token, the application service 
itself can't get a new delegation token for some specific user.  If your 
application service runs longer than the maximum renewable date of user's 
delegationToken,  one way is the user gets a new delegation token and your 
application service supports some mechanism to let user to update the 
delegation token and then refresh the token in that user's UGI's credential.  
Another way is to support proxy user privileges in your running application, 
refer to YARN-2704.   Are you in the correct way?


was (Author: hitliuyi):
[~marsishandsome],  Agree with Steve you need to move this to Hadoop common if 
the patch only contains change in common.  Before this, I think you may have a 
mistake about how to use delegation token.

Do you want to update the delegation token through 
{{FileSystem#addDelegationTokens}}?  It will not get new delegation token again 
if old one exists in the credentials, also it may be more complicate than what 
you think.
Actually I am curious about how you write your long running application. Is 
your application just an user client or running on YARN or a separate service?  
If your application is just one user client, I mean it's not a service which is 
acessed by many user clients, then you still need to user Kerberos instead of 
delegationToken, but if your application is a real service which serves user 
clients, then the delegation token is the right one. The delegation token is 
used in your service to access HDFS on behalf the user, usually your 
application service can renew the delegation token, the application service 
itself can't get a new delegation token for some specific user.  If your 
application service runs longer than the maximum renewable date of user's 
delegationToken,  one way is the user gets a new delegation token and your 
application service supports some mechanism to let user to update the 
delegation token and then refresh the token in that user's UGI's credential.  
Another way is to support proxy user privileges in your running application, 
refer to YARN-2704.   Are you in the correct way?

> 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 = "x...@abc.com"
> val creds1 = new org.apache.hadoop.security.Credentials()
> val ugi1 = 
> UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab)
>