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/10/15 09:24:48 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/master 7d798b3c7 -> 4a043126b


HBASE-21290 No need to instantiate BlockCache 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/4a043126
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4a043126
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4a043126

Branch: refs/heads/master
Commit: 4a043126b1bc9ef92ddbe04732ec5ac769d15b0f
Parents: 7d798b3
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Oct 11 19:34:45 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Mon Oct 15 17:21:17 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/CacheConfig.java      | 36 ++++++++++++++++----
 .../apache/hadoop/hbase/mob/MobCacheConfig.java |  5 +++
 .../hbase/regionserver/HRegionServer.java       |  7 ++--
 .../apache/hadoop/hbase/master/TestMaster.java  |  7 ++++
 4 files changed, 46 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/4a043126/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index ac1af91..147568e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -48,10 +48,8 @@ public class CacheConfig {
   /**
    * Disabled cache configuration
    */
-
   public static final CacheConfig DISABLED = new CacheConfig();
 
-
   /**
    * Configuration key to cache data blocks on read. Bloom blocks and index blocks are always be
    * cached if the block cache is enabled.
@@ -247,7 +245,11 @@ public class CacheConfig {
    * @param conf hbase configuration
    */
   public CacheConfig(Configuration conf) {
-    this(CacheConfig.instantiateBlockCache(conf),
+    this(conf, true);
+  }
+
+  public CacheConfig(Configuration conf, boolean enableBlockCache) {
+    this(conf, enableBlockCache,
         conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ),
         DEFAULT_IN_MEMORY, // This is a family-level setting so can't be set
                            // strictly from conf
@@ -257,11 +259,32 @@ public class CacheConfig {
         conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE),
         conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED),
         conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY, DEFAULT_PREFETCH_ON_OPEN),
-        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT)
-     );
+        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT));
     LOG.info("Created cacheConfig: " + this);
   }
 
+  private CacheConfig(Configuration conf, boolean enableBlockCache,
+      final boolean cacheDataOnRead, final boolean inMemory,
+      final boolean cacheDataOnWrite, final boolean cacheIndexesOnWrite,
+      final boolean cacheBloomsOnWrite, final boolean evictOnClose,
+      final boolean cacheDataCompressed, final boolean prefetchOnOpen,
+      final boolean dropBehindCompaction) {
+    if (enableBlockCache) {
+      this.blockCache = CacheConfig.instantiateBlockCache(conf);
+    } else {
+      this.blockCache = null;
+    }
+    this.cacheDataOnRead = cacheDataOnRead;
+    this.inMemory = inMemory;
+    this.cacheDataOnWrite = cacheDataOnWrite;
+    this.cacheIndexesOnWrite = cacheIndexesOnWrite;
+    this.cacheBloomsOnWrite = cacheBloomsOnWrite;
+    this.evictOnClose = evictOnClose;
+    this.cacheDataCompressed = cacheDataCompressed;
+    this.prefetchOnOpen = prefetchOnOpen;
+    this.dropBehindCompaction = dropBehindCompaction;
+  }
+
   /**
    * Create a block cache configuration with the specified cache and configuration parameters.
    * @param blockCache reference to block cache, null if completely disabled
@@ -274,11 +297,10 @@ public class CacheConfig {
    * @param evictOnClose whether blocks should be evicted when HFile is closed
    * @param cacheDataCompressed whether to store blocks as compressed in the cache
    * @param prefetchOnOpen whether to prefetch blocks upon open
-   * @param cacheDataInL1 If more than one cache tier deployed, if true, cache this column families
-   *          data blocks up in the L1 tier.
    * @param dropBehindCompaction indicate that we should set drop behind to true when open a store
    *          file reader for compaction
    */
+  @VisibleForTesting
   CacheConfig(final BlockCache blockCache,
       final boolean cacheDataOnRead, final boolean inMemory,
       final boolean cacheDataOnWrite, final boolean cacheIndexesOnWrite,

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a043126/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
index 2305eba..971bb92 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
@@ -42,6 +42,11 @@ public class MobCacheConfig extends CacheConfig {
     instantiateMobFileCache(conf);
   }
 
+  public MobCacheConfig(Configuration conf, boolean needBlockCache) {
+    super(conf, needBlockCache);
+    instantiateMobFileCache(conf);
+  }
+
   /**
    * Instantiates the MobFileCache.
    * @param conf The current configuration.

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a043126/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 d5e9832..08ba62b 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
@@ -590,8 +590,11 @@ public class HRegionServer extends HasThread implements
       Superusers.initialize(conf);
 
       regionServerAccounting = new RegionServerAccounting(conf);
-      cacheConfig = new CacheConfig(conf);
-      mobCacheConfig = new MobCacheConfig(conf);
+      boolean isMasterNotCarryTable =
+          this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf) && !LoadBalancer
+              .isSystemTablesOnlyOnMaster(conf);
+      cacheConfig = new CacheConfig(conf, !isMasterNotCarryTable);
+      mobCacheConfig = new MobCacheConfig(conf, !isMasterNotCarryTable);
       uncaughtExceptionHandler = new UncaughtExceptionHandler() {
         @Override
         public void uncaughtException(Thread t, Throwable e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/4a043126/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 81bdb02..85cdb95 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,6 +20,7 @@ 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,5 +264,11 @@ 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());
+  }
 }