Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-6325

[replication] Race in ReplicationSourceManager.init can initiate a failover even if the node is alive

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.90.6, 0.92.1, 0.94.0
    • 0.94.1, 0.95.0
    • None
    • None
    • Reviewed

    Description

      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:

      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)
      

      It seems to me that just refreshing otherRegionServers after getting the list of currentReplicators would be enough to fix this.

      Attachments

        1. HBASE-6325-0.92.patch
          4 kB
          Jean-Daniel Cryans
        2. HBASE-6325-0.92-v2.patch
          4 kB
          Jean-Daniel Cryans

        Activity

          People

            jdcryans Jean-Daniel Cryans
            jdcryans Jean-Daniel Cryans
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: