hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jason Lowe (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (YARN-7102) NM heartbeat stuck when responseId overflows MAX_INT
Date Fri, 03 Nov 2017 18:30:00 GMT

    [ https://issues.apache.org/jira/browse/YARN-7102?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16238148#comment-16238148
] 

Jason Lowe commented on YARN-7102:
----------------------------------

Thanks for updating the patch!

I'm not so sure these tests are timing out as much as they are calling System.exit and thus
not completing the test properly.  I was able to reproduce the problem with at least one of
the tests without this patch applied.  It's the same problem as reported in YARN-6647.  Here's
what I saw in the test output:
{noformat}
2017-11-03 13:09:49,045 ERROR [Thread[Thread-156,5,main]] recovery.RMStateStore (RMStateStore.java:notifyStoreOperationFailedInternal(1141))
- State store operation failed 
java.lang.InterruptedException
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:502)
        at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1406)
        at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:990)
        at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:910)
        at org.apache.curator.framework.imps.CuratorTransactionImpl.doOperation(CuratorTransactionImpl.java:159)
        at org.apache.curator.framework.imps.CuratorTransactionImpl.access$200(CuratorTransactionImpl.java:44)
        at org.apache.curator.framework.imps.CuratorTransactionImpl$2.call(CuratorTransactionImpl.java:129)
        at org.apache.curator.framework.imps.CuratorTransactionImpl$2.call(CuratorTransactionImpl.java:125)
        at org.apache.curator.RetryLoop.callWithRetry(RetryLoop.java:109)
        at org.apache.curator.framework.imps.CuratorTransactionImpl.commit(CuratorTransactionImpl.java:122)
        at org.apache.hadoop.util.curator.ZKCuratorManager$SafeTransaction.commit(ZKCuratorManager.java:403)
        at org.apache.hadoop.util.curator.ZKCuratorManager.safeCreate(ZKCuratorManager.java:347)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.storeRMDTMasterKeyState(ZKRMStateStore.java:1133)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:464)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:448)
        at org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
        at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
        at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
        at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:1109)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.storeRMDTMasterKey(RMStateStore.java:941)
        at org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager.storeNewMasterKey(RMDelegationTokenSecretManager.java:89)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.storeDelegationKey(AbstractDelegationTokenSecretManager.java:261)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.updateCurrentKey(AbstractDelegationTokenSecretManager.java:355)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.rollMasterKey(AbstractDelegationTokenSecretManager.java:375)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager$ExpiredTokenRemover.run(AbstractDelegationTokenSecretManager.java:677)
        at java.lang.Thread.run(Thread.java:745)
[...]
2017-11-03 13:09:49,047 WARN  [Thread[Thread-156,5,main]] event.AsyncDispatcher (AsyncDispatcher.java:handle(268))
- AsyncDispatcher thread interrupted
java.lang.InterruptedException
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220)
        at java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
        at java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:339)
        at org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:265)
        at org.apache.hadoop.yarn.event.DrainDispatcher$2.handle(DrainDispatcher.java:91)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.notifyStoreOperationFailedInternal(RMStateStore.java:1144)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.access$1500(RMStateStore.java:86)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:467)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:448)
        at org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
        at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
        at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
        at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:1109)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.storeRMDTMasterKey(RMStateStore.java:941)
        at org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager.storeNewMasterKey(RMDelegationTokenSecretManager.java:89)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.storeDelegationKey(AbstractDelegationTokenSecretManager.java:261)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.updateCurrentKey(AbstractDelegationTokenSecretManager.java:355)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.rollMasterKey(AbstractDelegationTokenSecretManager.java:375)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager$ExpiredTokenRemover.run(AbstractDelegationTokenSecretManager.java:677)
        at java.lang.Thread.run(Thread.java:745)
2017-11-03 13:09:49,048 ERROR [Thread[Thread-156,5,main]] security.RMDelegationTokenSecretManager
(RMDelegationTokenSecretManager.java:storeNewMasterKey(91)) - Error in storing master key
with KeyID: 2
2017-11-03 13:09:49,049 DEBUG [Thread[Thread-156,5,main]] util.ExitUtil (ExitUtil.java:terminate(209))
- Exiting with status 1: org.apache.hadoop.yarn.exceptions.YarnRuntimeException: java.lang.InterruptedException
1: org.apache.hadoop.yarn.exceptions.YarnRuntimeException: java.lang.InterruptedException
        at org.apache.hadoop.util.ExitUtil.terminate(ExitUtil.java:265)
        at org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager.storeNewMasterKey(RMDelegationTokenSecretManager.java:92)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.storeDelegationKey(AbstractDelegationTokenSecretManager.java:261)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.updateCurrentKey(AbstractDelegationTokenSecretManager.java:355)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager.rollMasterKey(AbstractDelegationTokenSecretManager.java:375)
        at org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager$ExpiredTokenRemover.run(AbstractDelegationTokenSecretManager.java:677)
        at java.lang.Thread.run(Thread.java:745)
Caused by: org.apache.hadoop.yarn.exceptions.YarnRuntimeException: java.lang.InterruptedException
        at org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:273)
        at org.apache.hadoop.yarn.event.DrainDispatcher$2.handle(DrainDispatcher.java:91)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.notifyStoreOperationFailedInternal(RMStateStore.java:1144)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.access$1500(RMStateStore.java:86)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:467)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$StoreRMDTMasterKeyTransition.transition(RMStateStore.java:448)
        at org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
        at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
        at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
        at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:1109)
        at org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.storeRMDTMasterKey(RMStateStore.java:941)
        at org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager.storeNewMasterKey(RMDelegationTokenSecretManager.java:89)
        ... 5 more
Caused by: java.lang.InterruptedException
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1220)
        at java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
        at java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:339)
        at org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:265)
        ... 17 more
2017-11-03 13:09:49,049 INFO  [Thread[Thread-156,5,main]] util.ExitUtil (ExitUtil.java:terminate(210))
- Exiting with status 1: org.apache.hadoop.yarn.exceptions.YarnRuntimeException: java.lang.InterruptedException
{noformat}

It would be good to verify these tests aren't failing in a new way due to this patch, but
it's important to note that they can fail with the same symptom even without this patch.

As far as the patch is concerned, I think preserving the last response from old to new node
is the right approach.  However calling setAndUpdateNodeHeartbeatResponse does not seem appropriate
here.  That will cause the RMNodeImpl to reprocess the heartbeat contents again which is not
desired.  Instead this should just be: newNode.latestNodeHeartBeatResponse = rmNode.getLastNodeHeartbeatResponse().


> NM heartbeat stuck when responseId overflows MAX_INT
> ----------------------------------------------------
>
>                 Key: YARN-7102
>                 URL: https://issues.apache.org/jira/browse/YARN-7102
>             Project: Hadoop YARN
>          Issue Type: Bug
>            Reporter: Botong Huang
>            Assignee: Botong Huang
>            Priority: Critical
>         Attachments: YARN-7102-branch-2.8.v10.patch, YARN-7102-branch-2.8.v11.patch,
YARN-7102-branch-2.8.v9.patch, YARN-7102-branch-2.v9.patch, YARN-7102-branch-2.v9.patch, YARN-7102-branch-2.v9.patch,
YARN-7102.v1.patch, YARN-7102.v12.patch, YARN-7102.v13.patch, YARN-7102.v2.patch, YARN-7102.v3.patch,
YARN-7102.v4.patch, YARN-7102.v5.patch, YARN-7102.v6.patch, YARN-7102.v7.patch, YARN-7102.v8.patch,
YARN-7102.v9.patch
>
>
> ResponseId overflow problem in NM-RM heartbeat. This is same as AM-RM heartbeat in YARN-6640,
please refer to YARN-6640 for details. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message