[ 
https://issues.apache.org/jira/browse/YARN-11819?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Abhey Rana updated YARN-11819:
------------------------------
    Description: 
We observed in our production environment that the jobs submitted with a RM 
delegation token were continually failing to start.

Upon further investigation we figured out the following Stack Trace as the 
culprit -
{code:java}
java.io.IOException: Failed to renew token: Kind: HDFS_DELEGATION_TOKEN, 
Service: ha-hdfs:prod-EMPTY-hbase4a, Ident: (token for xyz: 
HDFS_DELEGATION_TOKEN 
owner=hbase/hbase-dnds1-1-abc.ops.xyz....@hbase-abc.app.xyz.net, renewer=xyz, 
realUser=, issueDate=1744651400720, maxDate=1745256200720, 
sequenceNumber=2575348, masterKeyId=790)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:533)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$1800(DelegationTokenRenewer.java:83)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:1067)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:1044)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750){code}
We took a look at the code and figured out that as part of the 
handleAppSubmitEvent we do catch the IOException and do request a 
DelegationTokenRefresh but we only do that for 
DelegationTokenRenewerAppRecoverEvent

Code Pointer - 
{code:java}
            if (ioe instanceof SecretManager.InvalidToken
                && dttr.maxDate < Time.now()
                && evt instanceof DelegationTokenRenewerAppRecoverEvent
                && token.getKind().equals(HDFS_DELEGATION_KIND)) {
              LOG.info("Failed to renew hdfs token " + dttr
                  + " on recovery as it expired, requesting new hdfs token for "
                  + applicationId + ", user=" + evt.getUser(), ioe);
              requestNewHdfsDelegationTokenAsProxyUser(
                  Arrays.asList(applicationId), evt.getUser(),
                  evt.shouldCancelAtEnd());
              continue;
            }{code}
The idea is to add a or statement in the event check.

evt instanceof DelegationTokenRenewerAppRecoverEvent || evt instanceof 
DelegationTokenRenewerAppSubmitEvent

 

  was:
We observed in our production environment that the jobs submitted with a RM 
delegation token were continually failing to start.



Upon further investigation we figured out the following Stack Trace as the 
culprit -
{code}
java.io.IOException: Failed to renew token: Kind: HDFS_DELEGATION_TOKEN, 
Service: ha-hdfs:prod-EMPTY-hbase4a, Ident: (token for sfdc: 
HDFS_DELEGATION_TOKEN 
owner=hbase/hbase4a-dnds1-86-ia7.ops.sfdc....@hbase4a-ia7.app.sfdc.net, 
renewer=sfdc, realUser=, issueDate=1744651400720, maxDate=1745256200720, 
sequenceNumber=2575348, masterKeyId=790)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:533)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$1800(DelegationTokenRenewer.java:83)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:1067)
        at 
org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:1044)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:750){code}

We took a look at the code and figured out that as part of the 
handleAppSubmitEvent we do catch the IOException and do request a 
DelegationTokenRefresh but we only do that for 
DelegationTokenRenewerAppRecoverEvent

Code Pointer - 
{code:java}
            if (ioe instanceof SecretManager.InvalidToken
                && dttr.maxDate < Time.now()
                && evt instanceof DelegationTokenRenewerAppRecoverEvent
                && token.getKind().equals(HDFS_DELEGATION_KIND)) {
              LOG.info("Failed to renew hdfs token " + dttr
                  + " on recovery as it expired, requesting new hdfs token for "
                  + applicationId + ", user=" + evt.getUser(), ioe);
              requestNewHdfsDelegationTokenAsProxyUser(
                  Arrays.asList(applicationId), evt.getUser(),
                  evt.shouldCancelAtEnd());
              continue;
            }{code}

The idea is to add a or statement in the event check.

evt instanceof DelegationTokenRenewerAppRecoverEvent || evt instanceof 
DelegationTokenRenewerAppSubmitEvent

 


> Request a HDFS delegation token refresh even at 
> DelegationTokenRenewerAppSubmitEvent
> ------------------------------------------------------------------------------------
>
>                 Key: YARN-11819
>                 URL: https://issues.apache.org/jira/browse/YARN-11819
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: resourcemanager
>    Affects Versions: 3.3.6, 3.5.0, 3.4.2
>            Reporter: Abhey Rana
>            Assignee: Abhey Rana
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 3.5.0, 3.4.2
>
>   Original Estimate: 96h
>  Remaining Estimate: 96h
>
> We observed in our production environment that the jobs submitted with a RM 
> delegation token were continually failing to start.
> Upon further investigation we figured out the following Stack Trace as the 
> culprit -
> {code:java}
> java.io.IOException: Failed to renew token: Kind: HDFS_DELEGATION_TOKEN, 
> Service: ha-hdfs:prod-EMPTY-hbase4a, Ident: (token for xyz: 
> HDFS_DELEGATION_TOKEN 
> owner=hbase/hbase-dnds1-1-abc.ops.xyz....@hbase-abc.app.xyz.net, renewer=xyz, 
> realUser=, issueDate=1744651400720, maxDate=1745256200720, 
> sequenceNumber=2575348, masterKeyId=790)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:533)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$1800(DelegationTokenRenewer.java:83)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:1067)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:1044)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>       at java.lang.Thread.run(Thread.java:750){code}
> We took a look at the code and figured out that as part of the 
> handleAppSubmitEvent we do catch the IOException and do request a 
> DelegationTokenRefresh but we only do that for 
> DelegationTokenRenewerAppRecoverEvent
> Code Pointer - 
> {code:java}
>             if (ioe instanceof SecretManager.InvalidToken
>                 && dttr.maxDate < Time.now()
>                 && evt instanceof DelegationTokenRenewerAppRecoverEvent
>                 && token.getKind().equals(HDFS_DELEGATION_KIND)) {
>               LOG.info("Failed to renew hdfs token " + dttr
>                   + " on recovery as it expired, requesting new hdfs token 
> for "
>                   + applicationId + ", user=" + evt.getUser(), ioe);
>               requestNewHdfsDelegationTokenAsProxyUser(
>                   Arrays.asList(applicationId), evt.getUser(),
>                   evt.shouldCancelAtEnd());
>               continue;
>             }{code}
> The idea is to add a or statement in the event check.
> evt instanceof DelegationTokenRenewerAppRecoverEvent || evt instanceof 
> DelegationTokenRenewerAppSubmitEvent
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org

Reply via email to