You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jd...@apache.org on 2011/03/15 22:45:07 UTC

svn commit: r1081956 - in /hbase/branches/0.90: CHANGES.txt src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java

Author: jdcryans
Date: Tue Mar 15 21:45:06 2011
New Revision: 1081956

URL: http://svn.apache.org/viewvc?rev=1081956&view=rev
Log:
HBASE-3648  [replication] failover is sloppy with znodes
HBASE-3596  [replication] Wait a few seconds before transferring queues

Modified:
    hbase/branches/0.90/CHANGES.txt
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
    hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java

Modified: hbase/branches/0.90/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/CHANGES.txt?rev=1081956&r1=1081955&r2=1081956&view=diff
==============================================================================
--- hbase/branches/0.90/CHANGES.txt (original)
+++ hbase/branches/0.90/CHANGES.txt Tue Mar 15 21:45:06 2011
@@ -29,6 +29,7 @@ Release 0.90.2 - February 9th, 2011
    HBASE-3636  a bug about deciding whether this key is a new key for the
                ROWCOL bloomfilter (Liyin Tang via Stack)
    HBASE-3639  FSUtils.getRootDir should qualify path
+   HBASE-3648  [replication] failover is sloppy with znodes
 
   IMPROVEMENTS
    HBASE-3542  MultiGet methods in Thrift
@@ -47,6 +48,7 @@ Release 0.90.2 - February 9th, 2011
    HBASE-3448  RegionSplitter, utility class to manually split tables
    HBASE-3610  Improve RegionSplitter performance
    HBASE-3496  HFile CLI Improvements
+   HBASE-3596  [replication] Wait a few seconds before transferring queues
 
 
 Release 0.90.1 - February 9th, 2011

Modified: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java?rev=1081956&r1=1081955&r2=1081956&view=diff
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java (original)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java Tue Mar 15 21:45:06 2011
@@ -516,7 +516,18 @@ public class ReplicationZookeeper {
       String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
       ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
     } catch (KeeperException e) {
-      LOG.info("Failed lock other rs", e);
+      // This exception will pop up if the znode under which we're trying to
+      // create the lock is already deleted by another region server, meaning
+      // that the transfer already occurred.
+      // NoNode => transfer is done and znodes are already deleted
+      // NodeExists => lock znode already created by another RS
+      if (e instanceof KeeperException.NoNodeException ||
+          e instanceof KeeperException.NodeExistsException) {
+        LOG.info("Won't transfer the queue," +
+            " another RS took care of it because of: " + e.getMessage());
+      } else {
+        LOG.info("Failed lock other rs", e);
+      }
       return false;
     }
     return true;
@@ -597,10 +608,30 @@ public class ReplicationZookeeper {
    * @param znode
    */
   public void deleteRsQueues(String znode) {
+    String fullpath = ZKUtil.joinZNode(rsZNode, znode);
     try {
-      ZKUtil.deleteNodeRecursively(this.zookeeper,
-          ZKUtil.joinZNode(rsZNode, znode));
+      List<String> clusters =
+        ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
+      for (String cluster : clusters) {
+        // We'll delete it later
+        if (cluster.equals(RS_LOCK_ZNODE)) {
+          continue;
+        }
+        String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
+        ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
+      }
+      // Finish cleaning up
+      ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
     } catch (KeeperException e) {
+      if (e instanceof KeeperException.NoNodeException ||
+          e instanceof KeeperException.NotEmptyException) {
+        // Testing a special case where another region server was able to
+        // create a lock just after we deleted it, but then was also able to
+        // delete the RS znode before us or its lock znode is still there.
+        if (e.getPath().equals(fullpath)) {
+          return;
+        }
+      }
       this.abortable.abort("Failed delete of " + znode, e);
     }
   }

Modified: hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java?rev=1081956&r1=1081955&r2=1081956&view=diff
==============================================================================
--- hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java (original)
+++ hbase/branches/0.90/src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java Tue Mar 15 21:45:06 2011
@@ -76,6 +76,8 @@ public class ReplicationSourceManager {
   private final Path logDir;
   // Path to the hlog archive
   private final Path oldLogDir;
+  // The number of ms that we wait before moving znodes, HBASE-3596
+  private final long sleepBeforeFailover;
 
   /**
    * Creates a replication manager and sets the watch on all the other
@@ -105,6 +107,7 @@ public class ReplicationSourceManager {
     this.fs = fs;
     this.logDir = logDir;
     this.oldLogDir = oldLogDir;
+    this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
     this.zkHelper.registerRegionServerListener(
         new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
     List<String> otherRSs =
@@ -291,6 +294,14 @@ public class ReplicationSourceManager {
    * @param rsZnode
    */
   public void transferQueues(String rsZnode) {
+    // Wait a bit before transferring the queues, we may be shutting down.
+    // This sleep may not be enough in some cases.
+    try {
+      Thread.sleep(this.sleepBeforeFailover);
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting before transferring a queue.");
+      Thread.currentThread().interrupt();
+    }
     // We try to lock that rs' queue directory
     if (this.stopper.isStopped()) {
       LOG.info("Not transferring queue since we are shutting down");