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

Varun Saxena commented on YARN-3798:
------------------------------------

Just to elaborate further, this issue comes because of Zookeeper being in an 
inconsistent state. 
This is because one of the zookeeper instances goes down.

The application node doesnt exist because Zookeeper instance hasn't yet synced 
the application node.
Probably on first failure, we can try and make a call to {{sync()}} to get 
consistent data from zookeeper. Or we can catch the exception and fail 
job(After retries).
Because IMHO RM should not go down.
Thoughts ?

> RM shutdown with NoNode exception while updating appAttempt on zk
> -----------------------------------------------------------------
>
>                 Key: YARN-3798
>                 URL: https://issues.apache.org/jira/browse/YARN-3798
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: resourcemanager
>         Environment: Suse 11 Sp3
>            Reporter: Bibin A Chundatt
>            Assignee: Varun Saxena
>
> RM going down with NoNode exception during create of znode for appattempt
> *Please find the exception logs*
> {code}
> 2015-06-09 10:09:44,732 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore: 
> ZKRMStateStore Session connected
> 2015-06-09 10:09:44,732 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore: 
> ZKRMStateStore Session restored
> 2015-06-09 10:09:44,886 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore: 
> Exception while executing a ZK operation.
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
>       at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
>       at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:1405)
>       at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:1310)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:926)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithCheck(ZKRMStateStore.java:1101)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithRetries(ZKRMStateStore.java:1122)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:937)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.createWithRetries(ZKRMStateStore.java:970)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.updateApplicationAttemptStateInternal(ZKRMStateStore.java:671)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:275)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:260)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:837)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:900)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:895)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
>       at java.lang.Thread.run(Thread.java:745)
> 2015-06-09 10:09:44,887 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore: Maxed 
> out ZK retries. Giving up!
> 2015-06-09 10:09:44,887 ERROR 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore: Error 
> updating appAttempt: appattempt_1433764310492_7152_000001
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
>       at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
>       at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:1405)
>       at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:1310)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:926)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithCheck(ZKRMStateStore.java:1101)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithRetries(ZKRMStateStore.java:1122)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:937)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.createWithRetries(ZKRMStateStore.java:970)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.updateApplicationAttemptStateInternal(ZKRMStateStore.java:671)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:275)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:260)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:837)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:900)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:895)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
>       at java.lang.Thread.run(Thread.java:745)
> 2015-06-09 10:09:44,898 INFO 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore: Updating 
> info for app: application_1433764310492_7152
> 2015-06-09 10:09:44,898 FATAL 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Received a 
> org.apache.hadoop.yarn.server.resourcemanager.RMFatalEvent of type 
> STATE_STORE_OP_FAILED. Cause:
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode
>       at org.apache.zookeeper.KeeperException.create(KeeperException.java:115)
>       at org.apache.zookeeper.ZooKeeper.multiInternal(ZooKeeper.java:1405)
>       at org.apache.zookeeper.ZooKeeper.multi(ZooKeeper.java:1310)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:926)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$4.run(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithCheck(ZKRMStateStore.java:1101)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore$ZKAction.runWithRetries(ZKRMStateStore.java:1122)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:923)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.doStoreMultiWithRetries(ZKRMStateStore.java:937)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.createWithRetries(ZKRMStateStore.java:970)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore.updateApplicationAttemptStateInternal(ZKRMStateStore.java:671)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:275)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$UpdateAppAttemptTransition.transition(RMStateStore.java:260)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
>       at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.handleStoreEvent(RMStateStore.java:837)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:900)
>       at 
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore$ForwardingEventHandler.handle(RMStateStore.java:895)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
>       at 
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
>       at java.lang.Thread.run(Thread.java:745)
> 2015-06-09 10:09:44,920 INFO org.apache.hadoop.util.ExitUtil: Exiting with 
> status 1
> {code}
> Zk leader process down has happened almost at the same time 
> On startup of  zk process znode for application was available
> *Current*
> RM going down and Job failure
> *Expected*
>  Submitted Job can fail but RM shutdown i not required



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to