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 2013/05/07 18:08:03 UTC

svn commit: r1479960 - in /hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase: LocalHBaseCluster.java master/HMaster.java master/balancer/BalancerChore.java util/JVMClusterUtil.java

Author: tedyu
Date: Tue May  7 16:08:02 2013
New Revision: 1479960

URL: http://svn.apache.org/r1479960
Log:
HBASE-7066 Some HMaster coprocessor exceptions are being swallowed in try catch blocks - Revert


Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java?rev=1479960&r1=1479959&r2=1479960&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java Tue May  7 16:08:02 2013
@@ -426,7 +426,7 @@ public class LocalHBaseCluster {
   /**
    * Shut down the mini HBase cluster
    */
-  public void shutdown() throws IOException {
+  public void shutdown() {
     JVMClusterUtil.shutdown(this.masterThreads, this.regionThreads);
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1479960&r1=1479959&r2=1479960&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Tue May  7 16:08:02 2013
@@ -1263,7 +1263,7 @@ MasterServices, Server {
     return balancerCutoffTime;
   }
 
-  public boolean balance() throws IOException {
+  public boolean balance() {
     // if master not initialized, don't run balancer.
     if (!this.initialized) {
       LOG.debug("Master has not been initialized, don't run balancer.");
@@ -1291,8 +1291,13 @@ MasterServices, Server {
       }
 
       if (this.cpHost != null) {
-        if (this.cpHost.preBalance()) {
-          LOG.debug("Coprocessor bypassing balancer request");
+        try {
+          if (this.cpHost.preBalance()) {
+            LOG.debug("Coprocessor bypassing balancer request");
+            return false;
+          }
+        } catch (IOException ioe) {
+          LOG.error("Error invoking master coprocessor preBalance()", ioe);
           return false;
         }
       }
@@ -1330,7 +1335,12 @@ MasterServices, Server {
         }
       }
       if (this.cpHost != null) {
-        this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
+        try {
+          this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
+        } catch (IOException ioe) {
+          // balancing already succeeded so don't change the result
+          LOG.error("Error invoking master coprocessor postBalance()", ioe);
+        }
       }
     }
     return balancerRan;
@@ -1338,11 +1348,7 @@ MasterServices, Server {
 
   @Override
   public BalanceResponse balance(RpcController c, BalanceRequest request) throws ServiceException {
-    try {
-      return BalanceResponse.newBuilder().setBalancerRan(balance()).build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
+    return BalanceResponse.newBuilder().setBalancerRan(balance()).build();
   }
 
   enum BalanceSwitchMode {
@@ -2109,12 +2115,16 @@ MasterServices, Server {
     return rsFatals;
   }
 
-  public void shutdown() throws IOException {
+  public void shutdown() {
     if (spanReceiverHost != null) {
       spanReceiverHost.closeReceivers();
     }
     if (cpHost != null) {
-      cpHost.preShutdown();
+      try {
+        cpHost.preShutdown();
+      } catch (IOException ioe) {
+        LOG.error("Error call master coprocessor preShutdown()", ioe);
+      }
     }
     if (mxBean != null) {
       MBeanUtil.unregisterMBean(mxBean);
@@ -2134,17 +2144,17 @@ MasterServices, Server {
   @Override
   public ShutdownResponse shutdown(RpcController controller, ShutdownRequest request)
   throws ServiceException {
-    try {
-      shutdown();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
+    shutdown();
     return ShutdownResponse.newBuilder().build();
   }
 
-  public void stopMaster() throws IOException {
+  public void stopMaster() {
     if (cpHost != null) {
-      cpHost.preStopMaster();
+      try {
+        cpHost.preStopMaster();
+      } catch (IOException ioe) {
+        LOG.error("Error call master coprocessor preStopMaster()", ioe);
+      }
     }
     stop("Stopped by " + Thread.currentThread().getName());
   }
@@ -2152,11 +2162,7 @@ MasterServices, Server {
   @Override
   public StopMasterResponse stopMaster(RpcController controller, StopMasterRequest request)
   throws ServiceException {
-    try {
-      stopMaster();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
+    stopMaster();
     return StopMasterResponse.newBuilder().build();
   }
 

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java?rev=1479960&r1=1479959&r2=1479960&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BalancerChore.java Tue May  7 16:08:02 2013
@@ -18,21 +18,17 @@
 
 package org.apache.hadoop.hbase.master.balancer;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.Chore;
 import org.apache.hadoop.hbase.master.HMaster;
 
-import java.io.IOException;
-
 /**
  * Chore that will call HMaster.balance{@link org.apache.hadoop.hbase.master.HMaster#balance()} when
  * needed.
  */
 @InterfaceAudience.Private
 public class BalancerChore extends Chore {
-  private static final Log LOG = LogFactory.getLog(BalancerChore.class);
+
   private final HMaster master;
 
   public BalancerChore(HMaster master) {
@@ -44,10 +40,6 @@ public class BalancerChore extends Chore
 
   @Override
   protected void chore() {
-    try {
-      master.balance();
-    } catch (IOException ioe) {
-      LOG.error("Error invoking balancer", ioe);
-    }
+    master.balance();
   }
 }

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java?rev=1479960&r1=1479959&r2=1479960&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java Tue May  7 16:08:02 2013
@@ -233,7 +233,7 @@ public class JVMClusterUtil {
    * @param regionservers
    */
   public static void shutdown(final List<MasterThread> masters,
-      final List<RegionServerThread> regionservers) throws IOException {
+      final List<RegionServerThread> regionservers) {
     LOG.debug("Shutting down HBase Cluster");
     if (masters != null) {
       // Do backups first.
@@ -315,4 +315,4 @@ public class JVMClusterUtil {
       Thread.currentThread().interrupt();
     }
   }
-}
\ No newline at end of file
+}