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

Jean-Daniel Cryans commented on HBASE-6325:
-------------------------------------------

bq. Below returns true if we added regionservers. What if we are adding 
regionservers we already had in otherRegionServers (how do I know only a 
newRsList is returned? Because called on construction and nodeCreated?)

Maybe the method isn't clear enough, it really just drops the current list and 
recreates it from ZK and it doesn't really matter if we got new servers or if 
some are gone. It's one of those cases of we need to keep a copy in memory of a 
list that's in ZK.

bq. This synchronize is not needed anymore since its done inside in 
refreshOtherRegionServersList?

I didn't look at ArrayList's source code but it seems to me that toString would 
use an iterator to print all the values and since it's fail-fast it would throw 
an exception if, while it's printing, we receive a ZK event and it updates the 
list.
                
> [replication] Race in ReplicationSourceManager.init can initiate a failover 
> even if the node is alive
> -----------------------------------------------------------------------------------------------------
>
>                 Key: HBASE-6325
>                 URL: https://issues.apache.org/jira/browse/HBASE-6325
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 0.90.6, 0.92.1, 0.94.0
>            Reporter: Jean-Daniel Cryans
>            Assignee: Jean-Daniel Cryans
>             Fix For: 0.90.7, 0.92.2, 0.96.0, 0.94.2
>
>         Attachments: HBASE-6325-0.92.patch
>
>
> Yet another bug found during the leap second madness, it's possible to miss 
> the registration of new region servers so that in 
> ReplicationSourceManager.init we start the failover of a live and replicating 
> region server. I don't think there's data loss but the RS that's being failed 
> over will die on:
> {noformat}
> 2012-07-01 06:25:15,604 FATAL 
> org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server 
> sv4r23s48,10304,1341112194623: Writing replication status
> org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = 
> NoNode for 
> /hbase/replication/rs/sv4r23s48,10304,1341112194623/4/sv4r23s48%2C10304%2C1341112194623.1341112195369
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
>         at 
> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.setData(ZooKeeper.java:1246)
>         at 
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.setData(RecoverableZooKeeper.java:372)
>         at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:655)
>         at org.apache.hadoop.hbase.zookeeper.ZKUtil.setData(ZKUtil.java:697)
>         at 
> org.apache.hadoop.hbase.replication.ReplicationZookeeper.writeReplicationStatus(ReplicationZookeeper.java:470)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceManager.logPositionAndCleanOldLogs(ReplicationSourceManager.java:154)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.shipEdits(ReplicationSource.java:607)
>         at 
> org.apache.hadoop.hbase.replication.regionserver.ReplicationSource.run(ReplicationSource.java:368)
> {noformat}
> It seems to me that just refreshing {{otherRegionServers}} after getting the 
> list of {{currentReplicators}} would be enough to fix this.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to