[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2016-03-01 Thread steveloughran
Github user steveloughran commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r54569902 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +130,21 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-12-15 Thread andrewor14
Github user andrewor14 commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-164852995 If we're waiting on the HDFS JIRA to be resolved, can we close this PR for now? We can always re-open it later once the other issue is addressed. --- If your

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-12-15 Thread marsishandsome
Github user marsishandsome closed the pull request at: https://github.com/apache/spark/pull/9168 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-22 Thread steveloughran
Github user steveloughran commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42779515 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +130,21 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-22 Thread tgravescs
Github user tgravescs commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-150306708 So I talked to one of the token experts in HDFS and it should not require a token for each namenode in HA mode. Are you sure you don't have a setup issue?

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-22 Thread marsishandsome
Github user marsishandsome commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42826304 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +130,21 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-22 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-150403538 Let's wait for the HDFS JIRA https://issues.apache.org/jira/browse/HDFS-9276. --- If your project is set up for it, you can reply to this email and have your

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-21 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-150077142 The Hadoop Client Version I'm using is: 2.5.0-cdh5.2.0, which is packaged in spark assembly jar. I've update the code, using hadoop-1 compatible API now.

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-21 Thread tgravescs
Github user tgravescs commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149887452 which version of hadoop are you using and have you filed with Hadoop? --- If your project is set up for it, you can reply to this email and have your reply appear on

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-21 Thread steveloughran
Github user steveloughran commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42604617 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +130,21 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-21 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149814342 I've tested in both version 1.4.1 and 1.5.1. This patch works. --- If your project is set up for it, you can reply to this email and have your reply appear

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-20 Thread steveloughran
Github user steveloughran commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42482462 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +132,20 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-20 Thread steveloughran
Github user steveloughran commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149531922 solution 4: fix HDFS. Don't be afraid of filing reports & submitting patches there; it's a better long-term strategy than trying to work around things downstream.

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-20 Thread tgravescs
Github user tgravescs commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149581574 HDFS ha is supposed to be transparent and is supposed to use the same token as non-ha. It looks more like it actually isn't logging you in from the keytab or

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-20 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149765972 In non-HA mode, there's only one token for the name node, so this bug will not occure. --- If your project is set up for it, you can reply to this email and

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-20 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149788293 Hi all, I have updated the patch and only use Hadoop's public stable API. I will submit a patch to Hadoop. This patch is just a workaround and

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149172392 Can one of the admins verify this patch? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread marsishandsome
GitHub user marsishandsome opened a pull request: https://github.com/apache/spark/pull/9168 [SPARK-11182] HDFS Delegation Token will be expired when calling "UserGroupInformation.getCurrentUser.addCredentials" in HA mode In HA mode, DFSClient will generate HDFS Delegation Token for

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread harishreedharan
Github user harishreedharan commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42421617 --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala --- @@ -177,6 +177,7 @@ private[yarn] class

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread tgravescs
Github user tgravescs commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149343245 Can you please describe the scenario in detail? I assume if it fails over to HA namenode something doesn't work? --- If your project is set up for it, you can reply

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread harishreedharan
Github user harishreedharan commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149338159 /cc @tgravescs --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread harishreedharan
Github user harishreedharan commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42420896 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +132,20 @@ class SparkHadoopUtil extends Logging {

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149398746 There are several Solutions, all works: 1 set dfs.namenode.delegation.token.max-lifetime to a big value. 2 use the configuration --conf

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149394322 The Scenario is as follows: 1. Kerberos is enabled. 2. NameNode HA is enabled. 3. In order to test Token expired, I change the configuration

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread marsishandsome
Github user marsishandsome commented on the pull request: https://github.com/apache/spark/pull/9168#issuecomment-149398132 The reason to my opinion is: 1 Spark AM will get a HDFS Delegation Token and add it to the Current User's Credential. This Token looks like: token1:

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread marsishandsome
Github user marsishandsome commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42450902 --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala --- @@ -177,6 +177,7 @@ private[yarn] class

[GitHub] spark pull request: [SPARK-11182] HDFS Delegation Token will be ex...

2015-10-19 Thread steveloughran
Github user steveloughran commented on a diff in the pull request: https://github.com/apache/spark/pull/9168#discussion_r42392020 --- Diff: core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala --- @@ -130,6 +132,20 @@ class SparkHadoopUtil extends Logging {