You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zg...@apache.org on 2018/11/01 08:20:26 UTC

hbase git commit: HBASE-21388 No need to instantiate MemStoreLAB for master which not carry table

Repository: hbase
Updated Branches:
  refs/heads/master 91611cc7e -> 6bdaedd7c


HBASE-21388 No need to instantiate MemStoreLAB for master which not carry table


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

Branch: refs/heads/master
Commit: 6bdaedd7ce806d3ec3d5a9a4540b544594898374
Parents: 91611cc
Author: Guanghao Zhang <zg...@apache.org>
Authored: Mon Oct 29 11:50:53 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Thu Nov 1 16:15:18 2018 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/master/HMaster.java |  7 +-
 .../hadoop/hbase/regionserver/ChunkCreator.java |  3 +-
 .../hbase/regionserver/HRegionServer.java       |  3 +-
 .../apache/hadoop/hbase/master/TestMaster.java  |  7 --
 .../hbase/master/TestMasterNotCarryTable.java   | 90 ++++++++++++++++++++
 5 files changed, 98 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6bdaedd7/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 8eaa87d..1efffbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -915,8 +915,11 @@ public class HMaster extends HRegionServer implements MasterServices {
 
     this.masterActiveTime = System.currentTimeMillis();
     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
-    // Initialize the chunkCreator
-    initializeMemStoreChunkCreator();
+
+    // Only initialize the MemStoreLAB when master carry table
+    if (LoadBalancer.isTablesOnMaster(conf)) {
+      initializeMemStoreChunkCreator();
+    }
     this.fileSystemManager = new MasterFileSystem(conf);
     this.walManager = new MasterWalManager(this);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6bdaedd7/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
index 44ba65f..1e5e94f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChunkCreator.java
@@ -135,7 +135,8 @@ public class ChunkCreator {
     return instance;
   }
 
-  static ChunkCreator getInstance() {
+  @VisibleForTesting
+  public static ChunkCreator getInstance() {
     return instance;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6bdaedd7/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 68eb006..24743b9 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
@@ -598,8 +598,7 @@ public class HRegionServer extends HasThread implements
 
       regionServerAccounting = new RegionServerAccounting(conf);
       boolean isMasterNotCarryTable =
-          this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf) && !LoadBalancer
-              .isSystemTablesOnlyOnMaster(conf);
+          this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf);
       cacheConfig = new CacheConfig(conf, !isMasterNotCarryTable);
       mobCacheConfig = new MobCacheConfig(conf, !isMasterNotCarryTable);
       uncaughtExceptionHandler = new UncaughtExceptionHandler() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/6bdaedd7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index 7ba431d..af89cbc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.master;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -263,11 +262,5 @@ public class TestMaster {
     // Assert lock gets put in place again.
     assertTrue(fs.exists(hbckLockPath));
   }
-
-  @Test
-  public void testMasterBlockCache() {
-    // Master not carry table in default, so no need to instantiate block cache, too.
-    assertNull(TEST_UTIL.getMiniHBaseCluster().getMaster().getCacheConfig().getBlockCache());
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/6bdaedd7/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
new file mode 100644
index 0000000..75d9019
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({MasterTests.class, MediumTests.class})
+public class TestMasterNotCarryTable {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterNotCarryTable.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestMasterNotCarryTable.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static HMaster master;
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration c = UTIL.getConfiguration();
+    // We use local filesystem.  Set it so it writes into the testdir.
+    FSUtils.setRootDir(c, UTIL.getDataTestDir());
+    UTIL.startMiniZKCluster();
+    master = new HMaster(UTIL.getConfiguration());
+    master.start();
+    // As no regionservers, only wait master to create AssignmentManager.
+    while (master.getAssignmentManager() != null) {
+      LOG.debug("Wait master to create AssignmentManager");
+      Thread.sleep(1000);
+    }
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    master.stop("Shutdown");
+    UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testMasterNotCarryTable() {
+    // The default config is false
+    assertFalse(LoadBalancer.isTablesOnMaster(UTIL.getConfiguration()));
+    assertFalse(LoadBalancer.isSystemTablesOnlyOnMaster(UTIL.getConfiguration()));
+  }
+
+  @Test
+  public void testMasterBlockCache() {
+    // no need to instantiate block cache.
+    assertNull(master.getCacheConfig().getBlockCache());
+  }
+
+  @Test
+  public void testMasterMemStoreLAB() {
+    // no need to instantiate MemStoreLAB.
+    assertNull(ChunkCreator.getInstance());
+  }
+}