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

Himanshu Vashishtha updated HBASE-7440:
---------------------------------------

    Attachment: HBASE-7440-v2.patch

Using ZKUtil.createNodeIfNotExistsAndWatch as suggested by Lars. Didn't notice 
this API earlier!
Yes Matteo, the default value of znode is ENABLED, and this code is called 
while creating one. So, no check needed or such.
                
> ReplicationZookeeper#addPeer is racy
> ------------------------------------
>
>                 Key: HBASE-7440
>                 URL: https://issues.apache.org/jira/browse/HBASE-7440
>             Project: HBase
>          Issue Type: Bug
>          Components: Replication
>    Affects Versions: 0.94.3
>            Reporter: Himanshu Vashishtha
>            Assignee: Himanshu Vashishtha
>             Fix For: 0.96.0, 0.94.4
>
>         Attachments: HBASE-7440-v0.patch, HBASE-7440-v1.patch, 
> HBASE-7440-v2.patch
>
>
> While adding a peer, ReplicationZK does the znodes creation in three 
> transactions. Create :
> a) peers znode
> b) peerId specific znode, and
> c) peerState znode
> There is a PeerWatcher which invokes getPeer() (after steps b) and c)). If it 
> happens that while adding a peer, the control flows to getPeer() and step c) 
> has not been processed, it may results in a state where the peer will not be 
> added. This happens while running 
> TestMasterReplication#testCyclicReplication().
> {code}
> 2012-12-26 07:36:35,187 INFO  
> [RegionServer:0;p0120.XXXXX,38423,1356536179470-EventThread] 
> zookeeper.RecoverableZooKeeper(447): Node /2/replication/peers/1/peer-state 
> already exists and this is not a retry
> 2012-12-26 07:36:35,188 ERROR 
> [RegionServer:0;p0120.XXXXX,38423,1356536179470-EventThread] 
> regionserver.ReplicationSourceManager$PeersWatcher(527): Error while adding a 
> new peer
> org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode = 
> NodeExists for /2/replication/peers/1/peer-state
>       at org.apache.zookeeper.KeeperException.create(KeeperException.java:119)
>       at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>       at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:783)
>       at 
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.createNonSequential(RecoverableZooKeeper.java:428)
>       at 
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.create(RecoverableZooKeeper.java:410)
>       at 
> org.apache.hadoop.hbase.zookeeper.ZKUtil.createAndWatch(ZKUtil.java:1044)
>       at 
> org.apache.hadoop.hbase.replication.ReplicationPeer.startStateTracker(ReplicationPeer.java:82)
>       at 
> org.apache.hadoop.hbase.replication.ReplicationZookeeper.getPeer(ReplicationZookeeper.java:344)
>       at 
> org.apache.hadoop.hbase.replication.ReplicationZookeeper.connectToPeer(ReplicationZookeeper.java:307)
>       at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager$PeersWatcher.nodeChildrenChanged(ReplicationSourceManager.java:519)
>       at 
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:315)
>       at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:519)
>       at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:495)
> 2012-12-26 07:36:35,188 DEBUG 
> [RegionServer:0;p0120.XXXXX,55742,1356536171947-EventThread] 
> zookeeper.ZKUtil(1545): regionserver:55742-0x13bd7db39580004 Retrieved 36 
> byte(s) of data from znode /1/hbaseid; data=9ce66123-d3e8-4ae9-a249-afe03...
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to