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

Tarun Parimi updated YARN-10789:
--------------------------------
    Description: 
We are observing below error randomly during hadoop install and RM initial 
startup when HA is enabled and yarn.scheduler.configuration.store.class=zk is 
configured. This causes one of the RMs to not startup.

{code:java}
2021-05-26 12:59:18,986 INFO org.apache.hadoop.service.AbstractService: Service 
RMActiveServices failed in state INITED
org.apache.hadoop.service.ServiceStateException: java.io.IOException: 
org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = 
NodeExists for /confstore/CONF_STORE
{code}

We are trying to create the znode /confstore/CONF_STORE when we initialize the 
ZKConfigurationStore. But the problem is that the ZKConfigurationStore is 
initialized when CapacityScheduler does a serviceInit. This serviceInit is done 
by both Active and Standby RM. So we can run into a race condition when both 
Active and Standby try to create the same znode when both RM are started at 
same time.

ZKRMStateStore on the other hand avoids such race conditions, by creating the 
znodes only after serviceStart. serviceStart only happens for the active RM 
which won the leader election, unlike serviceInit which happens irrespective of 
leader election.

  was:
We are observing below error randomly during hadoop install and RM initial 
startup when HA is enabled and yarn.scheduler.configuration.store.class=zk is 
configured. This cause one of the RM's to not startup.

{code:java}
2021-05-26 12:59:18,986 INFO org.apache.hadoop.service.AbstractService: Service 
RMActiveServices failed in state INITED
org.apache.hadoop.service.ServiceStateException: java.io.IOException: 
org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = 
NodeExists for /confstore/CONF_STORE
{code}

We are trying to create the znode /confstore/CONF_STORE when we initialize the 
ZKConfigurationStore. But the problem is that the ZKConfigurationStore is 
initialized when CapacityScheduler does a serviceInit. This serviceInit is done 
by both Active and Standby RM. So we can run into a race condition when both 
Active and Standby try to create the same znode when both RM are started at 
same time.

ZKRMStateStore on the other hand avoids such race conditions, by creating the 
znodes only after serviceStart. serviceStart only happens for the active RM 
which won the leader election, unlike serviceInit which happens irrespective of 
leader election.


> RM HA startup can fail due to race conditions in ZKConfigurationStore
> ---------------------------------------------------------------------
>
>                 Key: YARN-10789
>                 URL: https://issues.apache.org/jira/browse/YARN-10789
>             Project: Hadoop YARN
>          Issue Type: Bug
>    Affects Versions: 3.0.0
>            Reporter: Tarun Parimi
>            Assignee: Tarun Parimi
>            Priority: Major
>
> We are observing below error randomly during hadoop install and RM initial 
> startup when HA is enabled and yarn.scheduler.configuration.store.class=zk is 
> configured. This causes one of the RMs to not startup.
> {code:java}
> 2021-05-26 12:59:18,986 INFO org.apache.hadoop.service.AbstractService: 
> Service RMActiveServices failed in state INITED
> org.apache.hadoop.service.ServiceStateException: java.io.IOException: 
> org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = 
> NodeExists for /confstore/CONF_STORE
> {code}
> We are trying to create the znode /confstore/CONF_STORE when we initialize 
> the ZKConfigurationStore. But the problem is that the ZKConfigurationStore is 
> initialized when CapacityScheduler does a serviceInit. This serviceInit is 
> done by both Active and Standby RM. So we can run into a race condition when 
> both Active and Standby try to create the same znode when both RM are started 
> at same time.
> ZKRMStateStore on the other hand avoids such race conditions, by creating the 
> znodes only after serviceStart. serviceStart only happens for the active RM 
> which won the leader election, unlike serviceInit which happens irrespective 
> of leader election.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
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