You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jx...@apache.org on 2013/02/01 00:11:01 UTC

svn commit: r1441252 - /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java

Author: jxiang
Date: Thu Jan 31 23:11:01 2013
New Revision: 1441252

URL: http://svn.apache.org/viewvc?rev=1441252&view=rev
Log:
HBASE-7730 HBaseAdmin#synchronousBalanceSwitch is not compatible with 0.92

Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=1441252&r1=1441251&r2=1441252&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Thu Jan 31 23:11:01 2013
@@ -23,6 +23,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.lang.reflect.Proxy;
+import java.lang.reflect.UndeclaredThrowableException;
 import java.net.SocketTimeoutException;
 import java.util.Arrays;
 import java.util.LinkedList;
@@ -89,7 +90,9 @@ public class HBaseAdmin implements Abort
   // want to wait a long time.
   private final int retryLongerMultiplier;
   private boolean aborted;
-  
+
+  private static volatile boolean synchronousBalanceSwitchSupported = true;
+
   /**
    * Constructor
    *
@@ -1477,10 +1480,21 @@ public class HBaseAdmin implements Abort
    */
   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
   throws MasterNotRunningException, ZooKeeperConnectionException {
-    if (synchronous == false) {
-      return balanceSwitch(on);
+    if (synchronous && synchronousBalanceSwitchSupported) {
+      try {
+        return getMaster().synchronousBalanceSwitch(on);
+      } catch (UndeclaredThrowableException ute) {
+        String error = ute.getCause().getMessage();
+        if (error != null && error.matches(
+            "(?s).+NoSuchMethodException:.+synchronousBalanceSwitch.+")) {
+          LOG.info("HMaster doesn't support synchronousBalanceSwitch");
+          synchronousBalanceSwitchSupported = false;
+        } else {
+          throw ute;
+        }
+      }
     }
-    return getMaster().synchronousBalanceSwitch(on);
+    return balanceSwitch(on);
   }
 
   /**