You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2016/07/05 20:50:17 UTC

hbase git commit: HBASE-16087 Replication shouldn't start on a master if if only hosts system tables

Repository: hbase
Updated Branches:
  refs/heads/branch-1.3 603decdbf -> 59c5900fa


HBASE-16087 Replication shouldn't start on a master if if only hosts system tables


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/59c5900f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/59c5900f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/59c5900f

Branch: refs/heads/branch-1.3
Commit: 59c5900fae4392b9a5fcca8dbf5543e1bea1e452
Parents: 603decd
Author: Elliott Clark <ec...@apache.org>
Authored: Wed Jun 22 15:40:28 2016 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Jul 5 13:48:28 2016 -0700

----------------------------------------------------------------------
 .../hbase/master/balancer/BaseLoadBalancer.java   | 13 +++++++++++++
 .../hadoop/hbase/regionserver/HRegionServer.java  | 18 ++++++++++++------
 2 files changed, 25 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/59c5900f/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
index b446172..c2529a8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/BaseLoadBalancer.java
@@ -1001,6 +1001,19 @@ public abstract class BaseLoadBalancer implements LoadBalancer {
     return tables != null && tables.length > 0;
   }
 
+  public static boolean userTablesOnMaster(Configuration conf) {
+    String[] tables = getTablesOnMaster(conf);
+    if (tables == null || tables.length == 0) {
+      return false;
+    }
+    for (String tn:tables) {
+      if (!tn.startsWith("hbase:")) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   @Override
   public void setConf(Configuration conf) {
     setSlop(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/59c5900f/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 2fe9200..0c6ee83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -106,6 +106,7 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.TableLockManager;
+import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.RequestConverter;
@@ -446,7 +447,7 @@ public class HRegionServer extends HasThread implements
   private RegionServerCoprocessorHost rsHost;
 
   private RegionServerProcedureManagerHost rspmHost;
-  
+
   private RegionServerQuotaManager rsQuotaManager;
 
   // Table level lock manager for locking for region operations
@@ -874,7 +875,7 @@ public class HRegionServer extends HasThread implements
 
     // Setup the Quota Manager
     rsQuotaManager = new RegionServerQuotaManager(this);
-    
+
     // Setup RPC client for master communication
     rpcClient = RpcClientFactory.createClient(conf, clusterId, new InetSocketAddress(
         rpcServices.isa.getAddress(), 0), clusterConnection.getConnectionMetrics());
@@ -943,7 +944,7 @@ public class HRegionServer extends HasThread implements
         // since the server is ready to run
         rspmHost.start();
       }
-      
+
       // Start the Quota Manager
       if (this.rsQuotaManager != null) {
         rsQuotaManager.start(getRpcServer().getScheduler());
@@ -1037,7 +1038,7 @@ public class HRegionServer extends HasThread implements
     if (rsQuotaManager != null) {
       rsQuotaManager.stop();
     }
-    
+
     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
     if (rspmHost != null) {
       rspmHost.stop(this.abortRequested || this.killed);
@@ -2598,7 +2599,7 @@ public class HRegionServer extends HasThread implements
   public ChoreService getChoreService() {
     return choreService;
   }
-  
+
   @Override
   public RegionServerQuotaManager getRegionServerQuotaManager() {
     return rsQuotaManager;
@@ -2620,6 +2621,11 @@ public class HRegionServer extends HasThread implements
       return;
     }
 
+    if ((server instanceof HMaster) &&
+        (!BaseLoadBalancer.userTablesOnMaster(conf))) {
+      return;
+    }
+
     // read in the name of the source replication class from the config file.
     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
@@ -2721,7 +2727,7 @@ public class HRegionServer extends HasThread implements
      }
      return tableRegions;
    }
-  
+
   /**
    * Gets the online tables in this RS.
    * This method looks at the in-memory onlineRegions.