You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2012/04/04 00:03:28 UTC

svn commit: r1309171 - in /hbase/branches/0.92: CHANGES.txt src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java

Author: tedyu
Date: Tue Apr  3 22:03:28 2012
New Revision: 1309171

URL: http://svn.apache.org/viewvc?rev=1309171&view=rev
Log:
HBASE-5606  SplitLogManger async delete node hangs log splitting when ZK connection is lost  
               (Prakash)

Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1309171&r1=1309170&r2=1309171&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Tue Apr  3 22:03:28 2012
@@ -24,7 +24,8 @@ Release 0.92.2 - Unreleased
                are overwritten by hbase-site.xml (Takuya Ueshin)
    HBASE-5535  Make the functions in task monitor synchronized (Liyin Tang)
    HBASE-5213  "hbase master stop" does not bring down backup masters (Gregory) 
-
+   HBASE-5606  SplitLogManger async delete node hangs log splitting when ZK connection is lost
+               (Prakash)
 
   IMPROVEMENTS
    HBASE-5592  Make it easier to get a table from shell (Ben West)

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java?rev=1309171&r1=1309170&r2=1309171&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/master/SplitLogManager.java Tue Apr  3 22:03:28 2012
@@ -394,6 +394,14 @@ public class SplitLogManager extends Zoo
     tot_mgr_get_data_queued.incrementAndGet();
   }
 
+  private void tryGetDataSetWatch(String path) {
+    // A negative retry count will lead to ignoring all error processing.
+    this.watcher.getRecoverableZooKeeper().getZooKeeper().
+        getData(path, this.watcher,
+        new GetDataAsyncCallback(), new Long(-1) /* retry count */);
+    tot_mgr_get_data_queued.incrementAndGet();
+  }
+
   private void getDataSetWatchSuccess(String path, byte[] data, int version) {
     if (data == null) {
       if (version == Integer.MIN_VALUE) {
@@ -892,11 +900,13 @@ public class SplitLogManager extends Zoo
         for (Map.Entry<String, Task> e : tasks.entrySet()) {
           String path = e.getKey();
           Task task = e.getValue();
-          // we have to do this check again because tasks might have
-          // been asynchronously assigned.
-          if (task.isUnassigned()) {
+          // we have to do task.isUnassigned() check again because tasks might
+          // have been asynchronously assigned. There is no locking required
+          // for these checks ... it is OK even if tryGetDataSetWatch() is
+          // called unnecessarily for a task
+          if (task.isUnassigned() && (task.status != FAILURE)) {
             // We just touch the znode to make sure its still there
-            getDataSetWatch(path, zkretries);
+            tryGetDataSetWatch(path);
           }
         }
         createRescanNode(Long.MAX_VALUE);
@@ -966,6 +976,12 @@ public class SplitLogManager extends Zoo
           return;
         }
         Long retry_count = (Long) ctx;
+
+        if (retry_count < 0) {
+          LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
+              path + ". Ignoring error. No error handling. No retrying.");
+          return;
+        }
         LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
             path + " retry=" + retry_count);
         if (retry_count == 0) {