You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by marsishandsome <gi...@git.apache.org> on 2015/10/19 11:57:32 UTC

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

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 each Name Node automatically, which will not be updated when Spark update Credentials for the current user.
    Spark should update these tokens in order to avoid Token Expired Error.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/marsishandsome/spark SPARK11182

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/9168.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #9168
    
----
commit 3bbfe61c74150e0de42573eaf736629164ccfe47
Author: guliangliang <gu...@qiyi.com>
Date:   2015-10-19T09:45:28Z

    [SPARK-11182] HDFS Delegation Token will be expired when calling UserGroupInformation.getCurrentUser.addCredentials in HA mode
    
    Change-Id: Ia1833198ef694dfbc5b560bddd1eef226012787b

----


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def addCredentialsToCurrentUser(credentials: Credentials, freshHadoopConf: Configuration): Unit ={
    +    UserGroupInformation.getCurrentUser.addCredentials(credentials)
    +
    +    // HACK:
    +    // In HA mode, the function FileSystem.addDelegationTokens only returns a token for HA
    +    // NameNode. HDFS Client will generate private tokens for each NameNode according to the
    +    // token for HA NameNode and uses these private tokens to communicate with each NameNode.
    +    // If spark only update token for HA NameNode, HDFS Client will use the old private tokens,
    +    // which will cause token expired Error.
    +    // So:
    +    // We create a new HDFS Client, so that the new HDFS Client will generate and update the
    +    // private tokens for each NameNode.
    +    FileSystem.get(freshHadoopConf).close()
    --- End diff --
    
    turns out that `FileSystem.newInstance()` can also be used. This is Hadoop 2.x+, so can't be used in code that's also designed to compile against Hadoop 1.x


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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: "ha-hdfs:hadoop-namenode" -> "Kind: HDFS_DELEGATION_TOKEN, Service: ha-hdfs:hadoop-namenode, Ident: (HDFS_DELEGATION_TOKEN token 328709 for test)".
    
    2 DFSClient will generate another 2 Tokens for each NameNode.
    token2: "ha-hdfs://xxx.xxx.xxx.xxx:8020" -> "Kind: HDFS_DELEGATION_TOKEN, Service: xxx.xxx.xxx.xxx:8020, Ident: (HDFS_DELEGATION_TOKEN token 328708 for test)"
    token3: "ha-hdfs://yyy:yyy:yyy:yyy:8020" -> "Kind: HDFS_DELEGATION_TOKEN, Service: yyy:yyy:yyy:yyy:8020, Ident: (HDFS_DELEGATION_TOKEN token 328708 for test)"
    
    3 DFSClient will not generate token2 and token3 automatically, when Spark update token1.
    DFSClient will only use token2 and token3 to communicate with the 2 Name Nodes.
    
    4 FileSystem has cache, calling FileSystem.get will get a cached DFSClient, which has old tokens.
    Spark only update token1, but DFSClient will use token2 and token3.



---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by tgravescs <gi...@git.apache.org>.
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 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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by tgravescs <gi...@git.apache.org>.
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 as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by harishreedharan <gi...@git.apache.org>.
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 this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by jzhuge <gi...@git.apache.org>.
Github user jzhuge commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    @marsishandsome and @steveloughran HDFS-9276 has been backported to Hadoop 2.8.2 (in branch-2.8.1 as well, but 2.8.1 became a security release).


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by tgravescs <gi...@git.apache.org>.
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 distributing the tokens.
    
    Have you tried without HA?  
    Did you verify the relogin from keytab and distribute happened?


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by jackiehff <gi...@git.apache.org>.
Github user jackiehff commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    @Tianny,we just upgrade our hadoop cluster version to 2.6.0-cdh5.11.2 then the error disappered!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by andrewor14 <gi...@git.apache.org>.
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 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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def addCredentialsToCurrentUser(credentials: Credentials, freshHadoopConf: Configuration): Unit ={
    +    UserGroupInformation.getCurrentUser.addCredentials(credentials)
    +
    +    // HACK:
    +    // In HA mode, the function FileSystem.addDelegationTokens only returns a token for HA
    +    // NameNode. HDFS Client will generate private tokens for each NameNode according to the
    +    // token for HA NameNode and uses these private tokens to communicate with each NameNode.
    +    // If spark only update token for HA NameNode, HDFS Client will use the old private tokens,
    +    // which will cause token expired Error.
    +    // So:
    +    // We create a new HDFS Client, so that the new HDFS Client will generate and update the
    +    // private tokens for each NameNode.
    +    FileSystem.get(freshHadoopConf).close()
    --- End diff --
    
    that's not going to create a new client; instead it goes to `get(getDefaultUri(conf), conf)`, which then caches it under a key of (fsURI + currentUser)...which is why Hadoop 2 added a way to explicitly get a unique one.
    
    To use this method to get a guaranteed unique instance, the code will need to set the bool to disable caching on that configuration , which is done by building up a property for that FS scheme:
    
    ```
    String disableCacheName = String.format("fs.%s.impl.disable.cache", scheme);
    conf.setBoolean(disableCacheName, true)
    ```
    
    A bit convoluted, but without it we'll get back the same filesystem instance, then stay stuck in the process.
    
    Note that the code to create a unique FS, say some method `createUniqueFSInstance(freshHadoopConf: Configuration): FileSystem` could be factored out as its own method, with a test to verify that repeated calls did return unique objects. That would validate the requirements and catch any regressions in the hadoop codebase in future.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def addCredentialsToCurrentUser(credentials: Credentials, freshHadoopConf: Configuration): Unit ={
    +    UserGroupInformation.getCurrentUser.addCredentials(credentials)
    +
    +    // HACK:
    +    // In HA mode, the function FileSystem.addDelegationTokens only returns a token for HA
    +    // NameNode. HDFS Client will generate private tokens for each NameNode according to the
    +    // token for HA NameNode and uses these private tokens to communicate with each NameNode.
    +    // If spark only update token for HA NameNode, HDFS Client will use the old private tokens,
    +    // which will cause token expired Error.
    +    // So:
    +    // We create a new HDFS Client, so that the new HDFS Client will generate and update the
    +    // private tokens for each NameNode.
    +    FileSystem.get(freshHadoopConf).close()
    --- End diff --
    
    Good Idea. I will refactor the patch after HDFS-9276 is fixed.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by harishreedharan <gi...@git.apache.org>.
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 AMDelegationTokenRenewer(
         })
         // Add the temp credentials back to the original ones.
         UserGroupInformation.getCurrentUser.addCredentials(tempCreds)
    +    SparkHadoopUtil.get.updateCurrentUserHDFSDelegationToken()
    --- End diff --
    
    So I don't understand this. We are already getting new tokens for what I assume is the primary namenode using the `obtainTokensForNamenodes` method call, correct? Is this only to get the tokens for the standby? 
    
     I don't think this will work though - You'd need to do the same thing we do for the primary (add the tokens to a new UGI and copy the credentials from there). From what I can remember, when you add new tokens for a service that already has tokens - the tokens won't get overwritten, even if they are expired - which is why we went with creating a new UGI and adding those credentials over (which would overwrite the old tokens). Are you sure the `HAUtil.cloneDelegationTokenForLogicalUri` will overwrite the tokens? 


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def updateCurrentUserHDFSDelegationToken(): Unit = {
    +    val conf = new Configuration()
    +    val nsId = DFSUtil.getNamenodeNameServiceId(conf)
    +    val isHaEnabled = HAUtil.isHAEnabled(conf, nsId)
    +
    +    if(isHaEnabled){
    +      val ugi = UserGroupInformation.getCurrentUser
    +      val uri = FileSystem.getDefaultUri(conf)
    +      val map = DFSUtil.getHaNnRpcAddresses(conf)
    --- End diff --
    
    I'd try and avoid this class if possible. It is tagged as `@Private` in hadoop, and in hadoop 2.8 it'll remain in the larger hadoop-hdfs JAR, not the leaner client-side hadoop-hdfs-client JAR. The method isn't going away, and there will be a {{DFSUtilClient}} where it will move to -but that doesn't help current code.
    
    If it can't be avoided, then the HDFS team will need to be kept up to date with its use and so encouraged not to break things


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 will be removed until the bug is fixed in Hadoop.



---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by Tianny <gi...@git.apache.org>.
Github user Tianny commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    @jackiehff Have you solved the problem?I met the error same as you.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    looking at the HDFS patch, it's in branch-2.9. We could backport to branch-2.8, though it's too late to get into the 2.8.0 RC


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by leocook <gi...@git.apache.org>.
Github user leocook commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    I just add principal&keytab conf when submit my job like blow:
    ```
    --principal=xxxx@yyyyyy \
    --keytab=aaaaa.keytab \
    ```
    
    And <code>spark.hadoop.fs.hdfs.impl.disable.cache=true</code> is not necessary\uff0cthe <code>fs.hdfs.impl.disable.cache</code> has already been set to true in <code>https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java</code>



---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def updateCurrentUserHDFSDelegationToken(): Unit = {
    +    val conf = new Configuration()
    +    val nsId = DFSUtil.getNamenodeNameServiceId(conf)
    +    val isHaEnabled = HAUtil.isHAEnabled(conf, nsId)
    +
    +    if(isHaEnabled){
    +      val ugi = UserGroupInformation.getCurrentUser
    +      val uri = FileSystem.getDefaultUri(conf)
    +      val map = DFSUtil.getHaNnRpcAddresses(conf)
    --- End diff --
    
    looking at `AMDelegationTokenRenewer`, theres a more fundamental issue with HDFS which spark is trying to work around. It should be checking for tokens being expired and issuing new ones, so avoiding the whole hoop-jumping exercise in `writeNewTokensToHDFS()`. Is there a JIRA for that issue? As that's clearly a problem broader than just spark. Get it fixed & backported to 2.7.x (maybe, 2.6.x) and the problem goes away.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 spark.hadoop.fs.hdfs.impl.disable.cache=true
    3 the patch I provide


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by jackiehff <gi...@git.apache.org>.
Github user jackiehff commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    @marsishandsome, the patch file in jira HDFS-9276 has been introduced into hadoop-2.6.0-cdh5.7.3 which is our hadoop cluster version, but I still got token can't be found in cache error when I running spark streaming job, so my question is whether spark source code also need to be modified according to your pull request? By the way, I also used the configuration " --conf spark.hadoop.fs.hdfs.impl.disable.cache=true", but it didn't work


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by tgravescs <gi...@git.apache.org>.
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?  
    
    I would like to see a Hadoop jira and comments from them on this before putting anything like this in.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 AMDelegationTokenRenewer(
         })
         // Add the temp credentials back to the original ones.
         UserGroupInformation.getCurrentUser.addCredentials(tempCreds)
    +    SparkHadoopUtil.get.updateCurrentUserHDFSDelegationToken()
    --- End diff --
    
    In HA mode, there are three tokens:
    1. ha token
    2. namenode1 token
    3. namenode2 token
    
    Spark only update ha token.
    HAUtil.cloneDelegationTokenForLogicalUri will copy ha token to namenode token.


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by AmplabJenkins <gi...@git.apache.org>.
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 project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by Tianny <gi...@git.apache.org>.
Github user Tianny commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    @jackiehff My Hadoop cluster version is 2.6.0-cdh5.13.1. But still have the problem. I'm working on it trying to upgrade the spark version to latest version 2.3.2. 
    
    By the way,I sent a email to you yesterday. Whatever, thx.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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.
    
    Please review the 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 project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 of the NameNode
    dfs.namenode.delegation.token.max-lifetime = 40min
    dfs.namenode.delegation.key.update-interval = 20min
    dfs.namenode.delegation.token.renew-interval = 20min
    
    4. The Spark Test Application will write a HDFS file every minute.
    
    5. Yarn Cluster Mode is used.
    
    6. --principal --keytab argument is used.
    
    
    After running 40 min, I got  the Error:
    15/10/16 16:09:19 ERROR ApplicationMaster: User class threw exception: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 12.0 failed 4 times, most recent failure: Lost task 0.3 in stage 12.0 (TID 30, node153-81-74-jylt.qiyi.hadoop): org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 324309 for test) is expired
            at org.apache.hadoop.ipc.Client.call(Client.java:1411)
            at org.apache.hadoop.ipc.Client.call(Client.java:1364)
            at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
            at com.sun.proxy.$Proxy14.create(Unknown Source)
            at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:287)
            at sun.reflect.GeneratedMethodAccessor45.invoke(Unknown Source)
            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:187)
            at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
            at com.sun.proxy.$Proxy15.create(Unknown Source)
            at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1645)
            at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1627)
            at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1552)
            at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:396)
            at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:392)
            at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
            at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:392)
            at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:336)
            at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:908)
            at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:801)
            at org.apache.hadoop.mapred.TextOutputFormat.getRecordWriter(TextOutputFormat.java:123)
            at org.apache.spark.SparkHadoopWriter.open(SparkHadoopWriter.scala:91)
            at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1104)
            at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1095)
            at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
            at org.apache.spark.scheduler.Task.run(Task.scala:88)
            at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
            at java.lang.Thread.run(Thread.java:745)



---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by harishreedharan <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def updateCurrentUserHDFSDelegationToken(): Unit = {
    +    val conf = new Configuration()
    +    val nsId = DFSUtil.getNamenodeNameServiceId(conf)
    +    val isHaEnabled = HAUtil.isHAEnabled(conf, nsId)
    +
    +    if(isHaEnabled){
    +      val ugi = UserGroupInformation.getCurrentUser
    +      val uri = FileSystem.getDefaultUri(conf)
    +      val map = DFSUtil.getHaNnRpcAddresses(conf)
    --- End diff --
    
    @steveloughran If this is not a good option, is there another way of doing this? Maybe look up a specific configuration? Considering the fact that the configuration params are not likely to change (since it would break compat), we could just directly use those?


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by steveloughran <gi...@git.apache.org>.
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 {
         UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename)
       }
     
    +  def addCredentialsToCurrentUser(credentials: Credentials, hadoopConf: Configuration): Unit ={
    +    UserGroupInformation.getCurrentUser.addCredentials(credentials)
    +
    +    // HACK:
    +    // In HA mode, the function FileSystem.addDelegationTokens only returns a token for HA
    +    // NameNode. HDFS Client will generate private tokens for each NameNode according to the
    +    // token for HA NameNode and uses these private tokens to communicate with each NameNode.
    +    // If spark only update token for HA NameNode, HDFS Client will use the old private tokens,
    +    // which will cause token expired Error.
    +    // So:
    +    // We create a new HDFS Client, so that the new HDFS Client will generate and update the
    +    // private tokens for each NameNode.
    +    FileSystem.newInstance(hadoopConf).close()
    --- End diff --
    
    It's good that there is a public way to do this, but someone needs to verify that `FileSystem#newInstance()` was backported to the version of Hadoop 1 which Spark support's, it went into Hadoop 0.21 with [HADOOP-4655](https://issues.apache.org/jira/browse/HADOOP-4655). A hadoop-1 build would show this.
    
    Abandoning Hadoop 1 support would simplify this...


---
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 feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #9168: [SPARK-11182] HDFS Delegation Token will be expired when ...

Posted by Tianny <gi...@git.apache.org>.
Github user Tianny commented on the issue:

    https://github.com/apache/spark/pull/9168
  
    > @Tianny,we just upgrade our hadoop cluster version to 2.6.0-cdh5.11.2 then the error disappered!
    
    My Hadoop cluster version is 2.6.0-cdh5.13.1. But still have the problem. I'm working on it trying to upgrade the spark version to latest version 2.3.2. 
    
    By the way,I sent a email to you yesterday. Whatever, thx.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


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

Posted by marsishandsome <gi...@git.apache.org>.
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 on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org