You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/12/31 12:45:52 UTC

[16/47] hbase git commit: HBASE-21514 Refactor CacheConfig

HBASE-21514 Refactor CacheConfig


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

Branch: refs/heads/HBASE-21512
Commit: 1971d02e725341fdee79b7ee2308a9870debe2f6
Parents: 68b5df0
Author: Guanghao Zhang <zg...@apache.org>
Authored: Thu Nov 29 10:30:45 2018 +0800
Committer: Guanghao Zhang <zg...@apache.org>
Committed: Tue Dec 18 13:43:30 2018 +0800

----------------------------------------------------------------------
 .../tmpl/regionserver/BlockCacheTmpl.jamon      |  36 +-
 .../tmpl/regionserver/BlockCacheViewTmpl.jamon  |   3 +-
 .../hbase/tmpl/regionserver/RSStatusTmpl.jamon  |   5 +-
 .../hbase/io/hfile/BlockCacheFactory.java       | 226 +++++++++
 .../hadoop/hbase/io/hfile/CacheConfig.java      | 499 ++++---------------
 .../hbase/io/hfile/CombinedBlockCache.java      |  12 +
 .../hadoop/hbase/io/hfile/HFileBlockIndex.java  |  17 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java  | 176 +++----
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java  |   9 +-
 .../hbase/io/hfile/bucket/BucketAllocator.java  |   4 +-
 .../assignment/MergeTableRegionsProcedure.java  |  15 +-
 .../assignment/SplitTableRegionProcedure.java   |  15 +-
 .../apache/hadoop/hbase/mob/MobCacheConfig.java |  64 ---
 .../apache/hadoop/hbase/mob/MobFileCache.java   |   5 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |  28 +-
 .../hadoop/hbase/regionserver/HRegion.java      |  40 +-
 .../hbase/regionserver/HRegionServer.java       |  55 +-
 .../hadoop/hbase/regionserver/HStore.java       |   2 +-
 .../hbase/regionserver/HeapMemoryManager.java   |  30 +-
 .../MetricsRegionServerWrapperImpl.java         | 267 +++-------
 .../hbase/regionserver/RSRpcServices.java       |   3 +-
 .../regionserver/RegionServerServices.java      |  15 +-
 .../hadoop/hbase/HBaseTestingUtility.java       |  48 +-
 .../hadoop/hbase/MockRegionServerServices.java  |  13 +
 ...estAvoidCellReferencesIntoShippedBlocks.java |   4 +-
 .../client/TestBlockEvictionFromClient.java     |  20 +-
 .../hadoop/hbase/client/TestFromClientSide.java |   3 +-
 .../hbase/io/encoding/TestEncodedSeekers.java   |  26 +-
 .../hbase/io/hfile/TestBlockCacheReporting.java |  47 +-
 .../hadoop/hbase/io/hfile/TestCacheConfig.java  |  53 +-
 .../hadoop/hbase/io/hfile/TestCacheOnWrite.java |  34 +-
 .../io/hfile/TestForceCacheImportantBlocks.java |  22 +-
 .../apache/hadoop/hbase/io/hfile/TestHFile.java |   9 +-
 .../hadoop/hbase/io/hfile/TestHFileBlock.java   |   3 +-
 .../hbase/io/hfile/TestHFileBlockIndex.java     |  10 +-
 .../hfile/TestLazyDataBlockDecompression.java   |  20 +-
 .../hadoop/hbase/io/hfile/TestPrefetch.java     |  22 +-
 .../io/hfile/TestScannerFromBucketCache.java    |  58 +--
 .../TestScannerSelectionUsingKeyRange.java      |   5 +-
 .../io/hfile/TestScannerSelectionUsingTTL.java  |  31 +-
 .../hadoop/hbase/master/MockRegionServer.java   |  13 +
 .../hbase/master/TestMasterNotCarryTable.java   |   8 +-
 .../hadoop/hbase/mob/TestMobFileCache.java      |  22 +-
 .../regionserver/DataBlockEncodingTool.java     |   7 +-
 .../EncodedSeekPerformanceTest.java             |   2 +-
 .../hbase/regionserver/TestAtomicOperation.java |  10 +-
 .../hbase/regionserver/TestBlocksRead.java      |  66 +--
 .../hbase/regionserver/TestBlocksScanned.java   |  38 +-
 .../regionserver/TestCacheOnWriteInSchema.java  |   6 +-
 .../regionserver/TestClearRegionBlockCache.java |  46 +-
 .../regionserver/TestCompoundBloomFilter.java   |  16 +-
 .../hbase/regionserver/TestHMobStore.java       |  84 ++--
 .../hbase/regionserver/TestHStoreFile.java      |  14 +-
 .../regionserver/TestMobStoreCompaction.java    |  20 +-
 .../regionserver/TestMultiColumnScanner.java    |  15 +-
 .../hbase/regionserver/TestRSStatusServlet.java |   9 +-
 .../hbase/regionserver/TestRecoveredEdits.java  |  38 +-
 .../regionserver/TestRowPrefixBloomFilter.java  |   2 +-
 .../regionserver/TestSecureBulkLoadManager.java |   2 +-
 59 files changed, 1096 insertions(+), 1276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
index 5ea5bcc..a18e6d4 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheTmpl.jamon
@@ -20,17 +20,12 @@ Template for rendering Block Cache tabs in RegionServer Status page.
 <%args>
 CacheConfig cacheConfig;
 Configuration config;
+BlockCache bc;
 </%args>
 <%java>
-  BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
-  String bcUrl = null;
-  String bcName = null;
-  if (bc != null) {
-    bcUrl = "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
-    bcName = bc.getClass().getSimpleName();
-  }
-  BlockCache [] bcs = cacheConfig == null? null: cacheConfig.getBlockCache() == null? null: cacheConfig.getBlockCache().getBlockCaches();
-  // If more than one bc, show evictions in each bc listing so can compare
+  String bcUrl = bc == null ? null : "http://hbase.apache.org/devapidocs/" + bc.getClass().getName().replaceAll("\\.", "/") + ".html";
+  String bcName = bc == null ? null : bc.getClass().getSimpleName();
+  BlockCache [] bcs = bc == null ? null : bc.getBlockCaches();
   boolean evictions = bcs != null && bcs.length > 1;
 </%java>
 <%import>
@@ -57,13 +52,13 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
     </ul>
     <div class="tab-content" style="padding-bottom: 9px; border-bottom: 1px solid #ddd;">
         <div class="tab-pane active" id="tab_bc_baseInfo">
-            <& bc_baseInfo; cacheConfig = cacheConfig; bcUrl = bcUrl; bcName = bcName;  &>
+            <& bc_baseInfo; bc = bc; bcUrl = bcUrl; bcName = bcName;  &>
         </div>
         <div class="tab-pane" id="tab_bc_config">
             <& bc_config; cacheConfig = cacheConfig &>
         </div>
         <div class="tab-pane" id="tab_bc_stats">
-            <& bc_stats; cacheConfig = cacheConfig &>
+            <& bc_stats; bc = bc &>
         </div>
         <div class="tab-pane" id="tab_bc_l1">
             <& bc_l; bc = bcs == null? bc: bcs[0]; name = "L1"; evictions = evictions; &>
@@ -76,12 +71,11 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
 <%def bc_baseInfo>
 <%args>
-    CacheConfig cacheConfig;
+    BlockCache bc;
     String bcUrl;
     String bcName;
 </%args>
 <%java>
-  BlockCache bc = cacheConfig == null? null: cacheConfig.getBlockCache();
   BlockCache [] bcs = bc == null? null: bc.getBlockCaches();
   String bcl1Url = null;
   String bcl1Name = null;
@@ -229,10 +223,10 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 
 <%def bc_stats>
 <%args>
-    CacheConfig cacheConfig;
+    BlockCache bc;
 </%args>
-<%if cacheConfig == null || cacheConfig.getBlockCache() == null %>
-<p>CacheConfig is null</p>
+<%if bc == null %>
+<p>BlockCache is null</p>
 <%else>
 <table class="table table-striped">
     <tr>
@@ -242,23 +236,23 @@ org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
     </tr>
     <tr>
         <td>Size</td>
-        <td><% TraditionalBinaryPrefix.long2String(cacheConfig.getBlockCache().getCurrentSize(),
+        <td><% TraditionalBinaryPrefix.long2String(bc.getCurrentSize(),
             "B", 1) %></td>
         <td>Current size of block cache in use</td>
     </tr>
     <tr>
         <td>Free</td>
-        <td><% TraditionalBinaryPrefix.long2String(cacheConfig.getBlockCache().getFreeSize(),
+        <td><% TraditionalBinaryPrefix.long2String(bc.getFreeSize(),
             "B", 1) %></td>
         <td>The total free memory currently available to store more cache entries</td>
     </tr>
     <tr>
         <td>Count</td>
-        <td><% String.format("%,d", cacheConfig.getBlockCache().getBlockCount()) %></td>
+        <td><% String.format("%,d", bc.getBlockCount()) %></td>
         <td>Number of blocks in block cache</td>
     </tr>
-    <& evictions_tmpl; bc = cacheConfig.getBlockCache(); &>
-    <& hits_tmpl; bc = cacheConfig.getBlockCache(); &>
+    <& evictions_tmpl; bc = bc; &>
+    <& hits_tmpl; bc = bc; &>
 </table>
 <p>If block cache is made up of more than one cache -- i.e. a L1 and a L2 -- then the above
 are combined counts. Request count is sum of hits and misses.</p>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
index c6d7a61..d5a9aa5 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/BlockCacheViewTmpl.jamon
@@ -23,6 +23,7 @@ CacheConfig cacheConfig;
 Configuration conf;
 String bcn;
 String bcv;
+BlockCache blockCache;
 </%args>
 <%import>
 java.util.*;
@@ -39,7 +40,7 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
 org.apache.hadoop.util.StringUtils;
 </%import>
 <%java>
-  BlockCache bc = cacheConfig == null ? null : cacheConfig.getBlockCache();
+  BlockCache bc = blockCache;
   BlockCache [] bcs = bc == null ? null : bc.getBlockCaches();
   if (bcn.equals("L1")) {
     bc = bcs == null || bcs.length == 0? bc: bcs[0];

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
index 646d835..e3c38b9 100644
--- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
+++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RSStatusTmpl.jamon
@@ -29,13 +29,14 @@ org.apache.hadoop.hbase.regionserver.HRegionServer;
 org.apache.hadoop.hbase.client.RegionInfo;
 org.apache.hadoop.hbase.ServerName;
 org.apache.hadoop.hbase.HBaseConfiguration;
+org.apache.hadoop.hbase.io.hfile.CacheConfig;
 org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
 org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 </%import>
 <%doc>If json AND bcn is NOT an empty string presume it a block cache view request.</%doc>
 <%if format.equals("json") && bcn != null && bcn.length() > 0  %>
-  <& BlockCacheViewTmpl; conf = regionServer.getConfiguration(); cacheConfig = regionServer.getCacheConfig(); bcn = bcn; bcv = bcv;  &>
+  <& BlockCacheViewTmpl; conf = regionServer.getConfiguration(); cacheConfig = new CacheConfig(regionServer.getConfiguration()); bcn = bcn; bcv = bcv; blockCache = regionServer.getBlockCache().orElse(null)  &>
   <%java return; %>
 <%elseif format.equals("json") %>
   <& ../common/TaskMonitorTmpl; filter = filter; format = "json" &>
@@ -109,7 +110,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
 
     <section>
     <h2>Block Cache</h2>
-    <& BlockCacheTmpl; cacheConfig = regionServer.getCacheConfig(); config = regionServer.getConfiguration() &>
+    <& BlockCacheTmpl; cacheConfig = new CacheConfig(regionServer.getConfiguration()); config = regionServer.getConfiguration(); bc = regionServer.getBlockCache().orElse(null) &>
     </section>
 
     <section>

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
new file mode 100644
index 0000000..98b3c4f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCacheFactory.java
@@ -0,0 +1,226 @@
+/**
+ * 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.io.hfile;
+
+import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY;
+import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
+import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public final class BlockCacheFactory {
+
+  private static final Logger LOG = LoggerFactory.getLogger(BlockCacheFactory.class.getName());
+
+  /**
+   * Configuration keys for Bucket cache
+   */
+
+  /**
+   * If the chosen ioengine can persist its state across restarts, the path to the file to persist
+   * to. This file is NOT the data file. It is a file into which we will serialize the map of
+   * what is in the data file. For example, if you pass the following argument as
+   * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"),
+   * <code>file:/tmp/bucketcache.data </code>, then we will write the bucketcache data to the file
+   * <code>/tmp/bucketcache.data</code> but the metadata on where the data is in the supplied file
+   * is an in-memory map that needs to be persisted across restarts. Where to store this
+   * in-memory state is what you supply here: e.g. <code>/tmp/bucketcache.map</code>.
+   */
+  public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY = "hbase.bucketcache.persistent.path";
+
+  public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads";
+
+  public static final String BUCKET_CACHE_WRITER_QUEUE_KEY = "hbase.bucketcache.writer.queuelength";
+
+  /**
+   * A comma-delimited array of values for use as bucket sizes.
+   */
+  public static final String BUCKET_CACHE_BUCKETS_KEY = "hbase.bucketcache.bucket.sizes";
+
+  /**
+   * Defaults for Bucket cache
+   */
+  public static final int DEFAULT_BUCKET_CACHE_WRITER_THREADS = 3;
+  public static final int DEFAULT_BUCKET_CACHE_WRITER_QUEUE = 64;
+
+  /**
+   * The target block size used by blockcache instances. Defaults to
+   * {@link HConstants#DEFAULT_BLOCKSIZE}.
+   * TODO: this config point is completely wrong, as it's used to determine the
+   * target block size of BlockCache instances. Rename.
+   */
+  public static final String BLOCKCACHE_BLOCKSIZE_KEY = "hbase.offheapcache.minblocksize";
+
+  private static final String EXTERNAL_BLOCKCACHE_KEY = "hbase.blockcache.use.external";
+  private static final boolean EXTERNAL_BLOCKCACHE_DEFAULT = false;
+
+  private static final String EXTERNAL_BLOCKCACHE_CLASS_KEY = "hbase.blockcache.external.class";
+
+  private BlockCacheFactory() {
+  }
+
+  public static BlockCache createBlockCache(Configuration conf) {
+    LruBlockCache onHeapCache = createOnHeapCache(conf);
+    if (onHeapCache == null) {
+      return null;
+    }
+    boolean useExternal = conf.getBoolean(EXTERNAL_BLOCKCACHE_KEY, EXTERNAL_BLOCKCACHE_DEFAULT);
+    if (useExternal) {
+      BlockCache l2CacheInstance = createExternalBlockcache(conf);
+      return l2CacheInstance == null ?
+          onHeapCache :
+          new InclusiveCombinedBlockCache(onHeapCache, l2CacheInstance);
+    } else {
+      // otherwise use the bucket cache.
+      BucketCache bucketCache = createBucketCache(conf);
+      if (!conf.getBoolean("hbase.bucketcache.combinedcache.enabled", true)) {
+        // Non combined mode is off from 2.0
+        LOG.warn(
+            "From HBase 2.0 onwards only combined mode of LRU cache and bucket cache is available");
+      }
+      return bucketCache == null ? onHeapCache : new CombinedBlockCache(onHeapCache, bucketCache);
+    }
+  }
+
+  private static LruBlockCache createOnHeapCache(final Configuration c) {
+    final long cacheSize = MemorySizeUtil.getOnHeapCacheSize(c);
+    if (cacheSize < 0) {
+      return null;
+    }
+    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
+    LOG.info(
+        "Allocating onheap LruBlockCache size=" + StringUtils.byteDesc(cacheSize) + ", blockSize="
+            + StringUtils.byteDesc(blockSize));
+    return new LruBlockCache(cacheSize, blockSize, true, c);
+  }
+
+  /**
+   * Enum of all built in external block caches.
+   * This is used for config.
+   */
+  private static enum ExternalBlockCaches {
+    memcached("org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache");
+    // TODO(eclark): Consider more. Redis, etc.
+    Class<? extends BlockCache> clazz;
+    ExternalBlockCaches(String clazzName) {
+      try {
+        clazz = (Class<? extends BlockCache>) Class.forName(clazzName);
+      } catch (ClassNotFoundException cnef) {
+        clazz = null;
+      }
+    }
+    ExternalBlockCaches(Class<? extends BlockCache> clazz) {
+      this.clazz = clazz;
+    }
+  }
+
+  private static BlockCache createExternalBlockcache(Configuration c) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Trying to use External l2 cache");
+    }
+    Class klass = null;
+
+    // Get the class, from the config. s
+    try {
+      klass = ExternalBlockCaches
+          .valueOf(c.get(EXTERNAL_BLOCKCACHE_CLASS_KEY, "memcache")).clazz;
+    } catch (IllegalArgumentException exception) {
+      try {
+        klass = c.getClass(EXTERNAL_BLOCKCACHE_CLASS_KEY, Class.forName(
+            "org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache"));
+      } catch (ClassNotFoundException e) {
+        return null;
+      }
+    }
+
+    // Now try and create an instance of the block cache.
+    try {
+      LOG.info("Creating external block cache of type: " + klass);
+      return (BlockCache) ReflectionUtils.newInstance(klass, c);
+    } catch (Exception e) {
+      LOG.warn("Error creating external block cache", e);
+    }
+    return null;
+
+  }
+
+  private static BucketCache createBucketCache(Configuration c) {
+    // Check for L2.  ioengine name must be non-null.
+    String bucketCacheIOEngineName = c.get(BUCKET_CACHE_IOENGINE_KEY, null);
+    if (bucketCacheIOEngineName == null || bucketCacheIOEngineName.length() <= 0) {
+      return null;
+    }
+
+    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
+    final long bucketCacheSize = MemorySizeUtil.getBucketCacheSize(c);
+    if (bucketCacheSize <= 0) {
+      throw new IllegalStateException("bucketCacheSize <= 0; Check " +
+          BUCKET_CACHE_SIZE_KEY + " setting and/or server java heap size");
+    }
+    if (c.get("hbase.bucketcache.percentage.in.combinedcache") != null) {
+      LOG.warn("Configuration 'hbase.bucketcache.percentage.in.combinedcache' is no longer "
+          + "respected. See comments in http://hbase.apache.org/book.html#_changes_of_note");
+    }
+    int writerThreads = c.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
+        DEFAULT_BUCKET_CACHE_WRITER_THREADS);
+    int writerQueueLen = c.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
+        DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
+    String persistentPath = c.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
+    String[] configuredBucketSizes = c.getStrings(BUCKET_CACHE_BUCKETS_KEY);
+    int [] bucketSizes = null;
+    if (configuredBucketSizes != null) {
+      bucketSizes = new int[configuredBucketSizes.length];
+      for (int i = 0; i < configuredBucketSizes.length; i++) {
+        int bucketSize = Integer.parseInt(configuredBucketSizes[i].trim());
+        if (bucketSize % 256 != 0) {
+          // We need all the bucket sizes to be multiples of 256. Having all the configured bucket
+          // sizes to be multiples of 256 will ensure that the block offsets within buckets,
+          // that are calculated, will also be multiples of 256.
+          // See BucketEntry where offset to each block is represented using 5 bytes (instead of 8
+          // bytes long). We would like to save heap overhead as less as possible.
+          throw new IllegalArgumentException("Illegal value: " + bucketSize + " configured for '"
+              + BUCKET_CACHE_BUCKETS_KEY + "'. All bucket sizes to be multiples of 256");
+        }
+        bucketSizes[i] = bucketSize;
+      }
+    }
+    BucketCache bucketCache = null;
+    try {
+      int ioErrorsTolerationDuration = c.getInt(
+          "hbase.bucketcache.ioengine.errors.tolerated.duration",
+          BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
+      // Bucket cache logs its stats on creation internal to the constructor.
+      bucketCache = new BucketCache(bucketCacheIOEngineName,
+          bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
+          ioErrorsTolerationDuration, c);
+    } catch (IOException ioex) {
+      LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
+    }
+    return bucketCache;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 a022552..cd9303d 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
@@ -17,26 +17,17 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_IOENGINE_KEY;
-import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY;
-
-import java.io.IOException;
+import java.util.Optional;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory;
-import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
-import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
-
 /**
  * Stores all of the cache objects and configuration for a single HFile.
  */
@@ -44,7 +35,6 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
 public class CacheConfig {
   private static final Logger LOG = LoggerFactory.getLogger(CacheConfig.class.getName());
 
-
   /**
    * Disabled cache configuration
    */
@@ -60,109 +50,38 @@ public class CacheConfig {
    * Configuration key to cache data blocks on write. There are separate
    * switches for bloom blocks and non-root index blocks.
    */
-  public static final String CACHE_BLOCKS_ON_WRITE_KEY =
-      "hbase.rs.cacheblocksonwrite";
+  public static final String CACHE_BLOCKS_ON_WRITE_KEY = "hbase.rs.cacheblocksonwrite";
 
   /**
    * Configuration key to cache leaf and intermediate-level index blocks on
    * write.
    */
-  public static final String CACHE_INDEX_BLOCKS_ON_WRITE_KEY =
-      "hfile.block.index.cacheonwrite";
+  public static final String CACHE_INDEX_BLOCKS_ON_WRITE_KEY = "hfile.block.index.cacheonwrite";
 
   /**
    * Configuration key to cache compound bloom filter blocks on write.
    */
-  public static final String CACHE_BLOOM_BLOCKS_ON_WRITE_KEY =
-      "hfile.block.bloom.cacheonwrite";
+  public static final String CACHE_BLOOM_BLOCKS_ON_WRITE_KEY = "hfile.block.bloom.cacheonwrite";
 
   /**
    * Configuration key to cache data blocks in compressed and/or encrypted format.
    */
-  public static final String CACHE_DATA_BLOCKS_COMPRESSED_KEY =
-      "hbase.block.data.cachecompressed";
+  public static final String CACHE_DATA_BLOCKS_COMPRESSED_KEY = "hbase.block.data.cachecompressed";
 
   /**
    * Configuration key to evict all blocks of a given file from the block cache
    * when the file is closed.
    */
-  public static final String EVICT_BLOCKS_ON_CLOSE_KEY =
-      "hbase.rs.evictblocksonclose";
-
-  /**
-   * Configuration keys for Bucket cache
-   */
-
-  /**
-   * If the chosen ioengine can persist its state across restarts, the path to the file to persist
-   * to. This file is NOT the data file. It is a file into which we will serialize the map of
-   * what is in the data file. For example, if you pass the following argument as
-   * BUCKET_CACHE_IOENGINE_KEY ("hbase.bucketcache.ioengine"),
-   * <code>file:/tmp/bucketcache.data </code>, then we will write the bucketcache data to the file
-   * <code>/tmp/bucketcache.data</code> but the metadata on where the data is in the supplied file
-   * is an in-memory map that needs to be persisted across restarts. Where to store this
-   * in-memory state is what you supply here: e.g. <code>/tmp/bucketcache.map</code>.
-   */
-  public static final String BUCKET_CACHE_PERSISTENT_PATH_KEY =
-      "hbase.bucketcache.persistent.path";
-
-  public static final String BUCKET_CACHE_WRITER_THREADS_KEY = "hbase.bucketcache.writer.threads";
-  public static final String BUCKET_CACHE_WRITER_QUEUE_KEY =
-      "hbase.bucketcache.writer.queuelength";
-
-  /**
-   * A comma-delimited array of values for use as bucket sizes.
-   */
-  public static final String BUCKET_CACHE_BUCKETS_KEY = "hbase.bucketcache.bucket.sizes";
+  public static final String EVICT_BLOCKS_ON_CLOSE_KEY = "hbase.rs.evictblocksonclose";
 
   /**
-   * Defaults for Bucket cache
-   */
-  public static final int DEFAULT_BUCKET_CACHE_WRITER_THREADS = 3;
-  public static final int DEFAULT_BUCKET_CACHE_WRITER_QUEUE = 64;
-
- /**
    * Configuration key to prefetch all blocks of a given file into the block cache
    * when the file is opened.
    */
-  public static final String PREFETCH_BLOCKS_ON_OPEN_KEY =
-      "hbase.rs.prefetchblocksonopen";
-
-  /**
-   * The target block size used by blockcache instances. Defaults to
-   * {@link HConstants#DEFAULT_BLOCKSIZE}.
-   * TODO: this config point is completely wrong, as it's used to determine the
-   * target block size of BlockCache instances. Rename.
-   */
-  public static final String BLOCKCACHE_BLOCKSIZE_KEY = "hbase.offheapcache.minblocksize";
-
-  private static final String EXTERNAL_BLOCKCACHE_KEY = "hbase.blockcache.use.external";
-  private static final boolean EXTERNAL_BLOCKCACHE_DEFAULT = false;
+  public static final String PREFETCH_BLOCKS_ON_OPEN_KEY = "hbase.rs.prefetchblocksonopen";
 
-  private static final String EXTERNAL_BLOCKCACHE_CLASS_KEY = "hbase.blockcache.external.class";
-  private static final String DROP_BEHIND_CACHE_COMPACTION_KEY =
+  public static final String DROP_BEHIND_CACHE_COMPACTION_KEY =
       "hbase.hfile.drop.behind.compaction";
-  private static final boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
-
-  /**
-   * Enum of all built in external block caches.
-   * This is used for config.
-   */
-  private static enum ExternalBlockCaches {
-    memcached("org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache");
-    // TODO(eclark): Consider more. Redis, etc.
-    Class<? extends BlockCache> clazz;
-    ExternalBlockCaches(String clazzName) {
-      try {
-        clazz = (Class<? extends BlockCache>) Class.forName(clazzName);
-      } catch (ClassNotFoundException cnef) {
-        clazz = null;
-      }
-    }
-    ExternalBlockCaches(Class<? extends BlockCache> clazz) {
-      this.clazz = clazz;
-    }
-  }
 
   // Defaults
   public static final boolean DEFAULT_CACHE_DATA_ON_READ = true;
@@ -173,9 +92,7 @@ public class CacheConfig {
   public static final boolean DEFAULT_EVICT_ON_CLOSE = false;
   public static final boolean DEFAULT_CACHE_DATA_COMPRESSED = false;
   public static final boolean DEFAULT_PREFETCH_ON_OPEN = false;
-
-  /** Local reference to the block cache, null if completely disabled */
-  private final BlockCache blockCache;
+  public static final boolean DROP_BEHIND_CACHE_COMPACTION_DEFAULT = true;
 
   /**
    * Whether blocks should be cached on read (default is on if there is a
@@ -183,7 +100,7 @@ public class CacheConfig {
    * If off we will STILL cache meta blocks; i.e. INDEX and BLOOM types.
    * This cannot be disabled.
    */
-  private boolean cacheDataOnRead;
+  private final boolean cacheDataOnRead;
 
   /** Whether blocks should be flagged as in-memory when being cached */
   private final boolean inMemory;
@@ -208,89 +125,54 @@ public class CacheConfig {
 
   private final boolean dropBehindCompaction;
 
-  /**
-   * Create a cache configuration using the specified configuration object and
-   * family descriptor.
-   * @param conf hbase configuration
-   * @param family column family configuration
-   */
-  public CacheConfig(Configuration conf, ColumnFamilyDescriptor family) {
-    this(GLOBAL_BLOCK_CACHE_INSTANCE,
-        conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ)
-           && family.isBlockCacheEnabled(),
-        family.isInMemory(),
-        // For the following flags we enable them regardless of per-schema settings
-        // if they are enabled in the global configuration.
-        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_DATA_ON_WRITE) || family.isCacheDataOnWrite(),
-        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_INDEXES_ON_WRITE) || family.isCacheIndexesOnWrite(),
-        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY,
-            DEFAULT_CACHE_BLOOMS_ON_WRITE) || family.isCacheBloomsOnWrite(),
-        conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY,
-            DEFAULT_EVICT_ON_CLOSE) || family.isEvictBlocksOnClose(),
-        conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED),
-        conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY,
-            DEFAULT_PREFETCH_ON_OPEN) || family.isPrefetchBlocksOnOpen(),
-        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT)
-     );
-    LOG.info("Created cacheConfig for " + family.getNameAsString() + ": " + this);
-  }
+  // Local reference to the block cache
+  private final BlockCache blockCache;
 
   /**
    * Create a cache configuration using the specified configuration object and
-   * defaults for family level settings. Only use if no column family context. Prefer
-   * {@link CacheConfig#CacheConfig(Configuration, ColumnFamilyDescriptor)}
-   * @see #CacheConfig(Configuration, ColumnFamilyDescriptor)
+   * defaults for family level settings. Only use if no column family context.
    * @param conf hbase configuration
    */
   public CacheConfig(Configuration conf) {
-    this(GLOBAL_BLOCK_CACHE_INSTANCE,
-        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
-        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE),
-        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_INDEXES_ON_WRITE),
-        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_BLOOMS_ON_WRITE),
-        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));
-    LOG.info("Created cacheConfig: " + this);
+    this(conf, null);
+  }
+
+  public CacheConfig(Configuration conf, BlockCache blockCache) {
+    this(conf, null, blockCache);
   }
 
   /**
-   * Create a block cache configuration with the specified cache and configuration parameters.
-   * @param blockCache reference to block cache, null if completely disabled
-   * @param cacheDataOnRead whether DATA blocks should be cached on read (we always cache INDEX
-   *          blocks and BLOOM blocks; this cannot be disabled).
-   * @param inMemory whether blocks should be flagged as in-memory
-   * @param cacheDataOnWrite whether data blocks should be cached on write
-   * @param cacheIndexesOnWrite whether index blocks should be cached on write
-   * @param cacheBloomsOnWrite whether blooms should be cached on write
-   * @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 dropBehindCompaction indicate that we should set drop behind to true when open a store
-   *          file reader for compaction
+   * Create a cache configuration using the specified configuration object and
+   * family descriptor.
+   * @param conf hbase configuration
+   * @param family column family configuration
    */
-  @VisibleForTesting
-  CacheConfig(final BlockCache blockCache,
-      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) {
+  public CacheConfig(Configuration conf, ColumnFamilyDescriptor family, BlockCache blockCache) {
+    this.cacheDataOnRead = conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ) &&
+        (family == null ? true : family.isBlockCacheEnabled());
+    this.inMemory = family == null ? DEFAULT_IN_MEMORY : family.isInMemory();
+    this.cacheDataCompressed =
+        conf.getBoolean(CACHE_DATA_BLOCKS_COMPRESSED_KEY, DEFAULT_CACHE_DATA_COMPRESSED);
+    this.dropBehindCompaction =
+        conf.getBoolean(DROP_BEHIND_CACHE_COMPACTION_KEY, DROP_BEHIND_CACHE_COMPACTION_DEFAULT);
+    // For the following flags we enable them regardless of per-schema settings
+    // if they are enabled in the global configuration.
+    this.cacheDataOnWrite =
+        conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE) ||
+            (family == null ? false : family.isCacheDataOnWrite());
+    this.cacheIndexesOnWrite =
+        conf.getBoolean(CACHE_INDEX_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_INDEXES_ON_WRITE) ||
+            (family == null ? false : family.isCacheIndexesOnWrite());
+    this.cacheBloomsOnWrite =
+        conf.getBoolean(CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_BLOOMS_ON_WRITE) ||
+            (family == null ? false : family.isCacheBloomsOnWrite());
+    this.evictOnClose = conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE) ||
+        (family == null ? false : family.isEvictBlocksOnClose());
+    this.prefetchOnOpen = conf.getBoolean(PREFETCH_BLOCKS_ON_OPEN_KEY, DEFAULT_PREFETCH_ON_OPEN) ||
+        (family == null ? false : family.isPrefetchBlocksOnOpen());
     this.blockCache = blockCache;
-    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;
+    LOG.info("Created cacheConfig: " + this + (family == null ? "" : " for family " + family) +
+        " with blockCache=" + blockCache);
   }
 
   /**
@@ -298,30 +180,29 @@ public class CacheConfig {
    * @param cacheConf
    */
   public CacheConfig(CacheConfig cacheConf) {
-    this(cacheConf.blockCache, cacheConf.cacheDataOnRead, cacheConf.inMemory,
-        cacheConf.cacheDataOnWrite, cacheConf.cacheIndexesOnWrite,
-        cacheConf.cacheBloomsOnWrite, cacheConf.evictOnClose,
-        cacheConf.cacheDataCompressed, cacheConf.prefetchOnOpen,
-        cacheConf.dropBehindCompaction);
+    this.cacheDataOnRead = cacheConf.cacheDataOnRead;
+    this.inMemory = cacheConf.inMemory;
+    this.cacheDataOnWrite = cacheConf.cacheDataOnWrite;
+    this.cacheIndexesOnWrite = cacheConf.cacheIndexesOnWrite;
+    this.cacheBloomsOnWrite = cacheConf.cacheBloomsOnWrite;
+    this.evictOnClose = cacheConf.evictOnClose;
+    this.cacheDataCompressed = cacheConf.cacheDataCompressed;
+    this.prefetchOnOpen = cacheConf.prefetchOnOpen;
+    this.dropBehindCompaction = cacheConf.dropBehindCompaction;
+    this.blockCache = cacheConf.blockCache;
   }
 
   private CacheConfig() {
-    this(null, false, false, false, false, false, false, false, false, false);
-  }
-
-  /**
-   * Checks whether the block cache is enabled.
-   */
-  public boolean isBlockCacheEnabled() {
-    return this.blockCache != null;
-  }
-
-  /**
-   * Returns the block cache.
-   * @return the block cache, or null if caching is completely disabled
-   */
-  public BlockCache getBlockCache() {
-    return this.blockCache;
+    this.cacheDataOnRead = false;
+    this.inMemory = false;
+    this.cacheDataOnWrite = false;
+    this.cacheIndexesOnWrite = false;
+    this.cacheBloomsOnWrite = false;
+    this.evictOnClose = false;
+    this.cacheDataCompressed = false;
+    this.prefetchOnOpen = false;
+    this.dropBehindCompaction = false;
+    this.blockCache = null;
   }
 
   /**
@@ -330,7 +211,7 @@ public class CacheConfig {
    * @return true if blocks should be cached on read, false if not
    */
   public boolean shouldCacheDataOnRead() {
-    return isBlockCacheEnabled() && cacheDataOnRead;
+    return cacheDataOnRead;
   }
 
   public boolean shouldDropBehindCompaction() {
@@ -343,20 +224,15 @@ public class CacheConfig {
    * available.
    */
   public boolean shouldCacheBlockOnRead(BlockCategory category) {
-    return isBlockCacheEnabled()
-        && (cacheDataOnRead ||
-            category == BlockCategory.INDEX ||
-            category == BlockCategory.BLOOM ||
-            (prefetchOnOpen &&
-                (category != BlockCategory.META &&
-                 category != BlockCategory.UNKNOWN)));
+    return cacheDataOnRead || category == BlockCategory.INDEX || category == BlockCategory.BLOOM ||
+        (prefetchOnOpen && (category != BlockCategory.META && category != BlockCategory.UNKNOWN));
   }
 
   /**
    * @return true if blocks in this file should be flagged as in-memory
    */
   public boolean isInMemory() {
-    return isBlockCacheEnabled() && this.inMemory;
+    return this.inMemory;
   }
 
   /**
@@ -364,11 +240,10 @@ public class CacheConfig {
    *         written, false if not
    */
   public boolean shouldCacheDataOnWrite() {
-    return isBlockCacheEnabled() && this.cacheDataOnWrite;
+    return this.cacheDataOnWrite;
   }
 
   /**
-   * Only used for testing.
    * @param cacheDataOnWrite whether data blocks should be written to the cache
    *                         when an HFile is written
    */
@@ -382,7 +257,7 @@ public class CacheConfig {
    *         is written, false if not
    */
   public boolean shouldCacheIndexesOnWrite() {
-    return isBlockCacheEnabled() && this.cacheIndexesOnWrite;
+    return this.cacheIndexesOnWrite;
   }
 
   /**
@@ -390,7 +265,7 @@ public class CacheConfig {
    *         is written, false if not
    */
   public boolean shouldCacheBloomsOnWrite() {
-    return isBlockCacheEnabled() && this.cacheBloomsOnWrite;
+    return this.cacheBloomsOnWrite;
   }
 
   /**
@@ -398,7 +273,7 @@ public class CacheConfig {
    *         reader is closed, false if not
    */
   public boolean shouldEvictOnClose() {
-    return isBlockCacheEnabled() && this.evictOnClose;
+    return this.evictOnClose;
   }
 
   /**
@@ -406,6 +281,7 @@ public class CacheConfig {
    * @param evictOnClose whether blocks should be evicted from the cache when an
    *                     HFile reader is closed
    */
+  @VisibleForTesting
   public void setEvictOnClose(boolean evictOnClose) {
     this.evictOnClose = evictOnClose;
   }
@@ -414,14 +290,13 @@ public class CacheConfig {
    * @return true if data blocks should be compressed in the cache, false if not
    */
   public boolean shouldCacheDataCompressed() {
-    return isBlockCacheEnabled() && this.cacheDataOnRead && this.cacheDataCompressed;
+    return this.cacheDataOnRead && this.cacheDataCompressed;
   }
 
   /**
    * @return true if this {@link BlockCategory} should be compressed in blockcache, false otherwise
    */
   public boolean shouldCacheCompressed(BlockCategory category) {
-    if (!isBlockCacheEnabled()) return false;
     switch (category) {
       case DATA:
         return this.cacheDataOnRead && this.cacheDataCompressed;
@@ -434,7 +309,7 @@ public class CacheConfig {
    * @return true if blocks should be prefetched into the cache on open, false if not
    */
   public boolean shouldPrefetchOnOpen() {
-    return isBlockCacheEnabled() && this.prefetchOnOpen;
+    return this.prefetchOnOpen;
   }
 
   /**
@@ -446,9 +321,6 @@ public class CacheConfig {
    * configuration.
    */
   public boolean shouldReadBlockFromCache(BlockType blockType) {
-    if (!isBlockCacheEnabled()) {
-      return false;
-    }
     if (cacheDataOnRead) {
       return true;
     }
@@ -462,7 +334,7 @@ public class CacheConfig {
       return true;
     }
     if (blockType.getCategory() == BlockCategory.BLOOM ||
-            blockType.getCategory() == BlockCategory.INDEX) {
+        blockType.getCategory() == BlockCategory.INDEX) {
       return true;
     }
     return false;
@@ -479,206 +351,21 @@ public class CacheConfig {
     return shouldCacheBlockOnRead(blockType.getCategory());
   }
 
-  @Override
-  public String toString() {
-    if (!isBlockCacheEnabled()) {
-      return "CacheConfig:disabled";
-    }
-    return "blockCache=" + getBlockCache() +
-      ", cacheDataOnRead=" + shouldCacheDataOnRead() +
-      ", cacheDataOnWrite=" + shouldCacheDataOnWrite() +
-      ", cacheIndexesOnWrite=" + shouldCacheIndexesOnWrite() +
-      ", cacheBloomsOnWrite=" + shouldCacheBloomsOnWrite() +
-      ", cacheEvictOnClose=" + shouldEvictOnClose() +
-      ", cacheDataCompressed=" + shouldCacheDataCompressed() +
-      ", prefetchOnOpen=" + shouldPrefetchOnOpen();
-  }
-
-  // Static block cache reference and methods
-
-  /**
-   * Static reference to the block cache, or null if no caching should be used
-   * at all.
-   */
-  // Clear this if in tests you'd make more than one block cache instance.
-  @VisibleForTesting
-  static BlockCache GLOBAL_BLOCK_CACHE_INSTANCE;
-  private static LruBlockCache ONHEAP_CACHE_INSTANCE = null;
-  private static BlockCache L2_CACHE_INSTANCE = null;// Can be BucketCache or External cache.
-
-  /** Boolean whether we have disabled the block cache entirely. */
-  @VisibleForTesting
-  static boolean blockCacheDisabled = false;
-
-  /**
-   * @param c Configuration to use.
-   * @return An L1 instance.  Currently an instance of LruBlockCache.
-   */
-  public static LruBlockCache getOnHeapCache(final Configuration c) {
-    return getOnHeapCacheInternal(c);
-  }
-
-  public CacheStats getOnHeapCacheStats() {
-    if (ONHEAP_CACHE_INSTANCE != null) {
-      return ONHEAP_CACHE_INSTANCE.getStats();
-    }
-    return null;
-  }
-
-  public CacheStats getL2CacheStats() {
-    if (L2_CACHE_INSTANCE != null) {
-      return L2_CACHE_INSTANCE.getStats();
-    }
-    return null;
-  }
-
-  /**
-   * @param c Configuration to use.
-   * @return An L1 instance.  Currently an instance of LruBlockCache.
-   */
-  private synchronized static LruBlockCache getOnHeapCacheInternal(final Configuration c) {
-    if (ONHEAP_CACHE_INSTANCE != null) {
-      return ONHEAP_CACHE_INSTANCE;
-    }
-    final long cacheSize = MemorySizeUtil.getOnHeapCacheSize(c);
-    if (cacheSize < 0) {
-      blockCacheDisabled = true;
-    }
-    if (blockCacheDisabled) return null;
-    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
-    LOG.info("Allocating onheap LruBlockCache size=" +
-      StringUtils.byteDesc(cacheSize) + ", blockSize=" + StringUtils.byteDesc(blockSize));
-    ONHEAP_CACHE_INSTANCE = new LruBlockCache(cacheSize, blockSize, true, c);
-    return ONHEAP_CACHE_INSTANCE;
-  }
-
-  private static BlockCache getExternalBlockcache(Configuration c) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Trying to use External l2 cache");
-    }
-    Class klass = null;
-
-    // Get the class, from the config. s
-    try {
-      klass = ExternalBlockCaches.valueOf(c.get(EXTERNAL_BLOCKCACHE_CLASS_KEY, "memcache")).clazz;
-    } catch (IllegalArgumentException exception) {
-      try {
-        klass = c.getClass(EXTERNAL_BLOCKCACHE_CLASS_KEY, Class.forName(
-            "org.apache.hadoop.hbase.io.hfile.MemcachedBlockCache"));
-      } catch (ClassNotFoundException e) {
-        return null;
-      }
-    }
-
-    // Now try and create an instance of the block cache.
-    try {
-      LOG.info("Creating external block cache of type: " + klass);
-      return (BlockCache) ReflectionUtils.newInstance(klass, c);
-    } catch (Exception e) {
-      LOG.warn("Error creating external block cache", e);
-    }
-    return null;
-
-  }
-
-  @VisibleForTesting
-  static BucketCache getBucketCache(Configuration c) {
-    // Check for L2.  ioengine name must be non-null.
-    String bucketCacheIOEngineName = c.get(BUCKET_CACHE_IOENGINE_KEY, null);
-    if (bucketCacheIOEngineName == null || bucketCacheIOEngineName.length() <= 0) return null;
-
-    int blockSize = c.getInt(BLOCKCACHE_BLOCKSIZE_KEY, HConstants.DEFAULT_BLOCKSIZE);
-    final long bucketCacheSize = MemorySizeUtil.getBucketCacheSize(c);
-    if (bucketCacheSize <= 0) {
-      throw new IllegalStateException("bucketCacheSize <= 0; Check " +
-        BUCKET_CACHE_SIZE_KEY + " setting and/or server java heap size");
-    }
-    if (c.get("hbase.bucketcache.percentage.in.combinedcache") != null) {
-      LOG.warn("Configuration 'hbase.bucketcache.percentage.in.combinedcache' is no longer "
-          + "respected. See comments in http://hbase.apache.org/book.html#_changes_of_note");
-    }
-    int writerThreads = c.getInt(BUCKET_CACHE_WRITER_THREADS_KEY,
-      DEFAULT_BUCKET_CACHE_WRITER_THREADS);
-    int writerQueueLen = c.getInt(BUCKET_CACHE_WRITER_QUEUE_KEY,
-      DEFAULT_BUCKET_CACHE_WRITER_QUEUE);
-    String persistentPath = c.get(BUCKET_CACHE_PERSISTENT_PATH_KEY);
-    String[] configuredBucketSizes = c.getStrings(BUCKET_CACHE_BUCKETS_KEY);
-    int [] bucketSizes = null;
-    if (configuredBucketSizes != null) {
-      bucketSizes = new int[configuredBucketSizes.length];
-      for (int i = 0; i < configuredBucketSizes.length; i++) {
-        int bucketSize = Integer.parseInt(configuredBucketSizes[i].trim());
-        if (bucketSize % 256 != 0) {
-          // We need all the bucket sizes to be multiples of 256. Having all the configured bucket
-          // sizes to be multiples of 256 will ensure that the block offsets within buckets,
-          // that are calculated, will also be multiples of 256.
-          // See BucketEntry where offset to each block is represented using 5 bytes (instead of 8
-          // bytes long). We would like to save heap overhead as less as possible.
-          throw new IllegalArgumentException("Illegal value: " + bucketSize + " configured for '"
-              + BUCKET_CACHE_BUCKETS_KEY + "'. All bucket sizes to be multiples of 256");
-        }
-        bucketSizes[i] = bucketSize;
-      }
-    }
-    BucketCache bucketCache = null;
-    try {
-      int ioErrorsTolerationDuration = c.getInt(
-        "hbase.bucketcache.ioengine.errors.tolerated.duration",
-        BucketCache.DEFAULT_ERROR_TOLERATION_DURATION);
-      // Bucket cache logs its stats on creation internal to the constructor.
-      bucketCache = new BucketCache(bucketCacheIOEngineName,
-        bucketCacheSize, blockSize, bucketSizes, writerThreads, writerQueueLen, persistentPath,
-        ioErrorsTolerationDuration, c);
-    } catch (IOException ioex) {
-      LOG.error("Can't instantiate bucket cache", ioex); throw new RuntimeException(ioex);
-    }
-    return bucketCache;
-  }
-
   /**
-   * Returns the block cache or <code>null</code> in case none should be used.
-   * Sets GLOBAL_BLOCK_CACHE_INSTANCE
+   * Returns the block cache.
    *
-   * @param conf  The current configuration.
-   * @return The block cache or <code>null</code>.
+   * @return the block cache, or null if caching is completely disabled
    */
-  public static synchronized BlockCache instantiateBlockCache(Configuration conf) {
-    if (GLOBAL_BLOCK_CACHE_INSTANCE != null) {
-      return GLOBAL_BLOCK_CACHE_INSTANCE;
-    }
-    if (blockCacheDisabled) {
-      return null;
-    }
-    LruBlockCache onHeapCache = getOnHeapCacheInternal(conf);
-    // blockCacheDisabled is set as a side-effect of getL1Internal(), so check it again after the
-    // call.
-    if (blockCacheDisabled) {
-      return null;
-    }
-    boolean useExternal = conf.getBoolean(EXTERNAL_BLOCKCACHE_KEY, EXTERNAL_BLOCKCACHE_DEFAULT);
-    if (useExternal) {
-      L2_CACHE_INSTANCE = getExternalBlockcache(conf);
-      GLOBAL_BLOCK_CACHE_INSTANCE = L2_CACHE_INSTANCE == null ? onHeapCache
-          : new InclusiveCombinedBlockCache(onHeapCache, L2_CACHE_INSTANCE);
-    } else {
-      // otherwise use the bucket cache.
-      L2_CACHE_INSTANCE = getBucketCache(conf);
-      if (!conf.getBoolean("hbase.bucketcache.combinedcache.enabled", true)) {
-        // Non combined mode is off from 2.0
-        LOG.warn(
-            "From HBase 2.0 onwards only combined mode of LRU cache and bucket cache is available");
-      }
-      GLOBAL_BLOCK_CACHE_INSTANCE = L2_CACHE_INSTANCE == null ? onHeapCache
-          : new CombinedBlockCache(onHeapCache, L2_CACHE_INSTANCE);
-    }
-    return GLOBAL_BLOCK_CACHE_INSTANCE;
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.ofNullable(this.blockCache);
   }
 
-  // Supposed to use only from tests. Some tests want to reinit the Global block cache instance
-  @VisibleForTesting
-  static synchronized void clearGlobalInstances() {
-    ONHEAP_CACHE_INSTANCE = null;
-    L2_CACHE_INSTANCE = null;
-    GLOBAL_BLOCK_CACHE_INSTANCE = null;
+  @Override
+  public String toString() {
+    return "cacheDataOnRead=" + shouldCacheDataOnRead() + ", cacheDataOnWrite="
+        + shouldCacheDataOnWrite() + ", cacheIndexesOnWrite=" + shouldCacheIndexesOnWrite()
+        + ", cacheBloomsOnWrite=" + shouldCacheBloomsOnWrite() + ", cacheEvictOnClose="
+        + shouldEvictOnClose() + ", cacheDataCompressed=" + shouldCacheDataCompressed()
+        + ", prefetchOnOpen=" + shouldPrefetchOnOpen();
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
index 5b17b38..b7b9c77 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java
@@ -152,6 +152,14 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
       this.bucketCacheStats = fcStats;
     }
 
+    public CacheStats getLruCacheStats() {
+      return this.lruCacheStats;
+    }
+
+    public CacheStats getBucketCacheStats() {
+      return this.bucketCacheStats;
+    }
+
     @Override
     public long getDataMissCount() {
       return lruCacheStats.getDataMissCount() + bucketCacheStats.getDataMissCount();
@@ -381,4 +389,8 @@ public class CombinedBlockCache implements ResizableBlockCache, HeapSize {
     return (this.l2Cache instanceof BucketCache)
         ? ((BucketCache) this.l2Cache).getRefCount(cacheKey) : 0;
   }
+
+  public LruBlockCache getOnHeapCache() {
+    return onHeapCache;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
index b91b48c..33b3f51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java
@@ -1103,9 +1103,11 @@ public class HFileBlockIndex {
           blockStream.write(midKeyMetadata);
         blockWriter.writeHeaderAndData(out);
         if (cacheConf != null) {
-          HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
-          cacheConf.getBlockCache().cacheBlock(new BlockCacheKey(nameForCaching,
-            rootLevelIndexPos, true, blockForCaching.getBlockType()), blockForCaching);
+          cacheConf.getBlockCache().ifPresent(cache -> {
+            HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
+            cache.cacheBlock(new BlockCacheKey(nameForCaching, rootLevelIndexPos, true,
+                blockForCaching.getBlockType()), blockForCaching);
+          });
         }
       }
 
@@ -1207,9 +1209,12 @@ public class HFileBlockIndex {
       blockWriter.writeHeaderAndData(out);
 
       if (getCacheOnWrite()) {
-        HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
-        cacheConf.getBlockCache().cacheBlock(new BlockCacheKey(nameForCaching,
-          beginOffset, true, blockForCaching.getBlockType()), blockForCaching);
+        cacheConf.getBlockCache().ifPresent(cache -> {
+          HFileBlock blockForCaching = blockWriter.getBlockForCaching(cacheConf);
+          cache.cacheBlock(
+              new BlockCacheKey(nameForCaching, beginOffset, true, blockForCaching.getBlockType()),
+              blockForCaching);
+        });
       }
 
       // Add intermediate index block size

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index a4a40ba..0fc9576 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -370,13 +370,16 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
 
   @Override
   public void returnBlock(HFileBlock block) {
-    BlockCache blockCache = this.cacheConf.getBlockCache();
-    if (blockCache != null && block != null) {
-      BlockCacheKey cacheKey = new BlockCacheKey(this.getFileContext().getHFileName(),
-          block.getOffset(), this.isPrimaryReplicaReader(), block.getBlockType());
-      blockCache.returnBlock(cacheKey, block);
+    if (block != null) {
+      this.cacheConf.getBlockCache().ifPresent(blockCache -> {
+        BlockCacheKey cacheKey =
+            new BlockCacheKey(this.getFileContext().getHFileName(), block.getOffset(),
+                this.isPrimaryReplicaReader(), block.getBlockType());
+        blockCache.returnBlock(cacheKey, block);
+      });
     }
   }
+
   /**
    * @return the first key in the file. May be null if file has no entries. Note
    *         that this is not the first row key, but rather the byte form of the
@@ -1293,69 +1296,67 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
    * Retrieve block from cache. Validates the retrieved block's type vs {@code expectedBlockType}
    * and its encoding vs. {@code expectedDataBlockEncoding}. Unpacks the block as necessary.
    */
-   private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
-       boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
-       DataBlockEncoding expectedDataBlockEncoding) throws IOException {
-     // Check cache for block. If found return.
-     if (cacheConf.isBlockCacheEnabled()) {
-       BlockCache cache = cacheConf.getBlockCache();
-       HFileBlock cachedBlock = (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock,
-         updateCacheMetrics);
-       if (cachedBlock != null) {
-         if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
-           HFileBlock compressedBlock = cachedBlock;
-           cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
-           // In case of compressed block after unpacking we can return the compressed block
+  private HFileBlock getCachedBlock(BlockCacheKey cacheKey, boolean cacheBlock, boolean useLock,
+      boolean isCompaction, boolean updateCacheMetrics, BlockType expectedBlockType,
+      DataBlockEncoding expectedDataBlockEncoding) throws IOException {
+    // Check cache for block. If found return.
+    if (cacheConf.getBlockCache().isPresent()) {
+      BlockCache cache = cacheConf.getBlockCache().get();
+      HFileBlock cachedBlock =
+          (HFileBlock) cache.getBlock(cacheKey, cacheBlock, useLock, updateCacheMetrics);
+      if (cachedBlock != null) {
+        if (cacheConf.shouldCacheCompressed(cachedBlock.getBlockType().getCategory())) {
+          HFileBlock compressedBlock = cachedBlock;
+          cachedBlock = compressedBlock.unpack(hfileContext, fsBlockReader);
+          // In case of compressed block after unpacking we can return the compressed block
           if (compressedBlock != cachedBlock) {
             cache.returnBlock(cacheKey, compressedBlock);
           }
         }
-         validateBlockType(cachedBlock, expectedBlockType);
-
-         if (expectedDataBlockEncoding == null) {
-           return cachedBlock;
-         }
-         DataBlockEncoding actualDataBlockEncoding =
-                 cachedBlock.getDataBlockEncoding();
-         // Block types other than data blocks always have
-         // DataBlockEncoding.NONE. To avoid false negative cache misses, only
-         // perform this check if cached block is a data block.
-         if (cachedBlock.getBlockType().isData() &&
-                 !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) {
-           // This mismatch may happen if a Scanner, which is used for say a
-           // compaction, tries to read an encoded block from the block cache.
-           // The reverse might happen when an EncodedScanner tries to read
-           // un-encoded blocks which were cached earlier.
-           //
-           // Because returning a data block with an implicit BlockType mismatch
-           // will cause the requesting scanner to throw a disk read should be
-           // forced here. This will potentially cause a significant number of
-           // cache misses, so update so we should keep track of this as it might
-           // justify the work on a CompoundScanner.
-           if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) &&
-                   !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) {
-             // If the block is encoded but the encoding does not match the
-             // expected encoding it is likely the encoding was changed but the
-             // block was not yet evicted. Evictions on file close happen async
-             // so blocks with the old encoding still linger in cache for some
-             // period of time. This event should be rare as it only happens on
-             // schema definition change.
-             LOG.info("Evicting cached block with key " + cacheKey +
-                     " because of a data block encoding mismatch" +
-                     "; expected: " + expectedDataBlockEncoding +
-                     ", actual: " + actualDataBlockEncoding);
-             // This is an error scenario. so here we need to decrement the
-             // count.
-             cache.returnBlock(cacheKey, cachedBlock);
-             cache.evictBlock(cacheKey);
-           }
-           return null;
-         }
-         return cachedBlock;
-       }
-     }
-     return null;
-   }
+        validateBlockType(cachedBlock, expectedBlockType);
+
+        if (expectedDataBlockEncoding == null) {
+          return cachedBlock;
+        }
+        DataBlockEncoding actualDataBlockEncoding = cachedBlock.getDataBlockEncoding();
+        // Block types other than data blocks always have
+        // DataBlockEncoding.NONE. To avoid false negative cache misses, only
+        // perform this check if cached block is a data block.
+        if (cachedBlock.getBlockType().isData() &&
+            !actualDataBlockEncoding.equals(expectedDataBlockEncoding)) {
+          // This mismatch may happen if a Scanner, which is used for say a
+          // compaction, tries to read an encoded block from the block cache.
+          // The reverse might happen when an EncodedScanner tries to read
+          // un-encoded blocks which were cached earlier.
+          //
+          // Because returning a data block with an implicit BlockType mismatch
+          // will cause the requesting scanner to throw a disk read should be
+          // forced here. This will potentially cause a significant number of
+          // cache misses, so update so we should keep track of this as it might
+          // justify the work on a CompoundScanner.
+          if (!expectedDataBlockEncoding.equals(DataBlockEncoding.NONE) &&
+              !actualDataBlockEncoding.equals(DataBlockEncoding.NONE)) {
+            // If the block is encoded but the encoding does not match the
+            // expected encoding it is likely the encoding was changed but the
+            // block was not yet evicted. Evictions on file close happen async
+            // so blocks with the old encoding still linger in cache for some
+            // period of time. This event should be rare as it only happens on
+            // schema definition change.
+            LOG.info("Evicting cached block with key " + cacheKey +
+                " because of a data block encoding mismatch" + "; expected: " +
+                expectedDataBlockEncoding + ", actual: " + actualDataBlockEncoding);
+            // This is an error scenario. so here we need to decrement the
+            // count.
+            cache.returnBlock(cacheKey, cachedBlock);
+            cache.evictBlock(cacheKey);
+          }
+          return null;
+        }
+        return cachedBlock;
+      }
+    }
+    return null;
+  }
 
   /**
    * @param metaBlockName
@@ -1391,26 +1392,24 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         this.isPrimaryReplicaReader(), BlockType.META);
 
       cacheBlock &= cacheConf.shouldCacheBlockOnRead(BlockType.META.getCategory());
-      if (cacheConf.isBlockCacheEnabled()) {
-        HFileBlock cachedBlock = getCachedBlock(cacheKey, cacheBlock, false, true, true,
-          BlockType.META, null);
-        if (cachedBlock != null) {
-          assert cachedBlock.isUnpacked() : "Packed block leak.";
-          // Return a distinct 'shallow copy' of the block,
-          // so pos does not get messed by the scanner
-          return cachedBlock;
-        }
-        // Cache Miss, please load.
+      HFileBlock cachedBlock =
+          getCachedBlock(cacheKey, cacheBlock, false, true, true, BlockType.META, null);
+      if (cachedBlock != null) {
+        assert cachedBlock.isUnpacked() : "Packed block leak.";
+        // Return a distinct 'shallow copy' of the block,
+        // so pos does not get messed by the scanner
+        return cachedBlock;
       }
+      // Cache Miss, please load.
 
       HFileBlock metaBlock = fsBlockReader.readBlockData(metaBlockOffset, blockSize, true, false).
           unpack(hfileContext, fsBlockReader);
 
       // Cache the block
       if (cacheBlock) {
-        cacheConf.getBlockCache().cacheBlock(cacheKey, metaBlock, cacheConf.isInMemory());
+        cacheConf.getBlockCache()
+            .ifPresent(cache -> cache.cacheBlock(cacheKey, metaBlock, cacheConf.isInMemory()));
       }
-
       return metaBlock;
     }
   }
@@ -1491,11 +1490,13 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
         BlockType.BlockCategory category = hfileBlock.getBlockType().getCategory();
 
         // Cache the block if necessary
-        if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) {
-          cacheConf.getBlockCache().cacheBlock(cacheKey,
-            cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked,
-            cacheConf.isInMemory());
-        }
+        cacheConf.getBlockCache().ifPresent(cache -> {
+          if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) {
+            cache.cacheBlock(cacheKey,
+                cacheConf.shouldCacheCompressed(category) ? hfileBlock : unpacked,
+                cacheConf.isInMemory());
+          }
+        });
 
         if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
           HFile.DATABLOCK_READ_COUNT.increment();
@@ -1569,13 +1570,14 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
   @Override
   public void close(boolean evictOnClose) throws IOException {
     PrefetchExecutor.cancel(path);
-    if (evictOnClose && cacheConf.isBlockCacheEnabled()) {
-      int numEvicted = cacheConf.getBlockCache().evictBlocksByHfileName(name);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("On close, file=" + name + " evicted=" + numEvicted
-          + " block(s)");
+    cacheConf.getBlockCache().ifPresent(cache -> {
+      if (evictOnClose) {
+        int numEvicted = cache.evictBlocksByHfileName(name);
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("On close, file=" + name + " evicted=" + numEvicted + " block(s)");
+        }
       }
-    }
+    });
     fsBlockReader.closeStreams();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index cfc3dd9..2726977 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -542,10 +542,11 @@ public class HFileWriterImpl implements HFile.Writer {
    *          the cache key.
    */
   private void doCacheOnWrite(long offset) {
-    HFileBlock cacheFormatBlock = blockWriter.getBlockForCaching(cacheConf);
-    cacheConf.getBlockCache().cacheBlock(
-        new BlockCacheKey(name, offset, true, cacheFormatBlock.getBlockType()),
-        cacheFormatBlock);
+    cacheConf.getBlockCache().ifPresent(cache -> {
+      HFileBlock cacheFormatBlock = blockWriter.getBlockForCaching(cacheConf);
+      cache.cacheBlock(new BlockCacheKey(name, offset, true, cacheFormatBlock.getBlockType()),
+          cacheFormatBlock);
+    });
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
index b8e4a0d..6caa975 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketAllocator.java
@@ -29,11 +29,11 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.atomic.LongAdder;
 
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.BucketEntry;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 
@@ -451,7 +451,7 @@ public final class BucketAllocator {
     BucketSizeInfo bsi = roundUpToBucketSizeInfo(blockSize);
     if (bsi == null) {
       throw new BucketAllocatorException("Allocation too big size=" + blockSize +
-        "; adjust BucketCache sizes " + CacheConfig.BUCKET_CACHE_BUCKETS_KEY +
+        "; adjust BucketCache sizes " + BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY +
         " to accomodate if size seems reasonable and you want it cached.");
     }
     long offset = bsi.allocateBlock();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index 7811d9b..5d84165 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -638,17 +638,18 @@ public class MergeTableRegionsProcedure
     final Configuration conf = env.getMasterConfiguration();
     final TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
 
-    for (String family: regionFs.getFamilies()) {
+    for (String family : regionFs.getFamilies()) {
       final ColumnFamilyDescriptor hcd = htd.getColumnFamily(Bytes.toBytes(family));
       final Collection<StoreFileInfo> storeFiles = regionFs.getStoreFiles(family);
 
       if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          // Create reference file(s) of the region in mergedDir
-          regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(),
-              storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true),
-            mergedDir);
+        for (StoreFileInfo storeFileInfo : storeFiles) {
+          // Create reference file(s) of the region in mergedDir.
+          // As this procedure is running on master, use CacheConfig.DISABLED means
+          // don't cache any block.
+          regionFs.mergeStoreFile(mergedRegion, family,
+              new HStoreFile(mfs.getFileSystem(), storeFileInfo, conf, CacheConfig.DISABLED,
+                  hcd.getBloomFilterType(), true), mergedDir);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
index b66d91f..9894e25 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java
@@ -638,16 +638,17 @@ public class SplitTableRegionProcedure
 
     TableDescriptor htd = env.getMasterServices().getTableDescriptors().get(getTableName());
     // Split each store file.
-    for (Map.Entry<String, Collection<StoreFileInfo>>e: files.entrySet()) {
-      byte [] familyName = Bytes.toBytes(e.getKey());
+    for (Map.Entry<String, Collection<StoreFileInfo>> e : files.entrySet()) {
+      byte[] familyName = Bytes.toBytes(e.getKey());
       final ColumnFamilyDescriptor hcd = htd.getColumnFamily(familyName);
       final Collection<StoreFileInfo> storeFiles = e.getValue();
       if (storeFiles != null && storeFiles.size() > 0) {
-        final CacheConfig cacheConf = new CacheConfig(conf, hcd);
-        for (StoreFileInfo storeFileInfo: storeFiles) {
-          StoreFileSplitter sfs =
-              new StoreFileSplitter(regionFs, familyName, new HStoreFile(mfs.getFileSystem(),
-                  storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true));
+        for (StoreFileInfo storeFileInfo : storeFiles) {
+          // As this procedure is running on master, use CacheConfig.DISABLED means
+          // don't cache any block.
+          StoreFileSplitter sfs = new StoreFileSplitter(regionFs, familyName,
+              new HStoreFile(mfs.getFileSystem(), storeFileInfo, conf, CacheConfig.DISABLED,
+                  hcd.getBloomFilterType(), true));
           futures.add(threadPool.submit(sfs));
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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
deleted file mode 100644
index 2305eba..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobCacheConfig.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- *
- * 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.mob;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-
-/**
- * The cache configuration for the mob.
- */
-@InterfaceAudience.Private
-public class MobCacheConfig extends CacheConfig {
-
-  private static MobFileCache mobFileCache;
-
-  public MobCacheConfig(Configuration conf, ColumnFamilyDescriptor family) {
-    super(conf, family);
-    instantiateMobFileCache(conf);
-  }
-
-  public MobCacheConfig(Configuration conf) {
-    super(conf);
-    instantiateMobFileCache(conf);
-  }
-
-  /**
-   * Instantiates the MobFileCache.
-   * @param conf The current configuration.
-   * @return The current instance of MobFileCache.
-   */
-  public static synchronized MobFileCache instantiateMobFileCache(Configuration conf) {
-    if (mobFileCache == null) {
-      mobFileCache = new MobFileCache(conf);
-    }
-    return mobFileCache;
-  }
-
-  /**
-   * Gets the MobFileCache.
-   * @return The MobFileCache.
-   */
-  public MobFileCache getMobFileCache() {
-    return mobFileCache;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
index b709d06..b987607 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFileCache.java
@@ -34,10 +34,11 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.util.IdLock;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.util.IdLock;
 
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -201,7 +202,7 @@ public class MobFileCache {
    * @return A opened mob file.
    * @throws IOException
    */
-  public MobFile openFile(FileSystem fs, Path path, MobCacheConfig cacheConf) throws IOException {
+  public MobFile openFile(FileSystem fs, Path path, CacheConfig cacheConf) throws IOException {
     if (!isCacheEnabled) {
       MobFile mobFile = MobFile.create(fs, path, conf, cacheConf);
       mobFile.open();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index 740eb08..b657183 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -48,9 +48,9 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.CorruptHFileException;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobFile;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobFileName;
 import org.apache.hadoop.hbase.mob.MobStoreEngine;
 import org.apache.hadoop.hbase.mob.MobUtils;
@@ -80,7 +80,7 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public class HMobStore extends HStore {
   private static final Logger LOG = LoggerFactory.getLogger(HMobStore.class);
-  private MobCacheConfig mobCacheConfig;
+  private MobFileCache mobFileCache;
   private Path homePath;
   private Path mobFamilyPath;
   private AtomicLong cellsCountCompactedToMob = new AtomicLong();
@@ -107,7 +107,7 @@ public class HMobStore extends HStore {
       final Configuration confParam) throws IOException {
     super(region, family, confParam);
     this.family = family;
-    this.mobCacheConfig = (MobCacheConfig) cacheConf;
+    this.mobFileCache = region.getMobFileCache();
     this.homePath = MobUtils.getMobHome(conf);
     this.mobFamilyPath = MobUtils.getMobFamilyPath(conf, this.getTableName(),
         family.getNameAsString());
@@ -126,14 +126,6 @@ public class HMobStore extends HStore {
   }
 
   /**
-   * Creates the mob cache config.
-   */
-  @Override
-  protected void createCacheConf(ColumnFamilyDescriptor family) {
-    cacheConf = new MobCacheConfig(conf, family);
-  }
-
-  /**
    * Gets current config.
    */
   public Configuration getConfiguration() {
@@ -256,7 +248,7 @@ public class HMobStore extends HStore {
       long maxKeyCount, Compression.Algorithm compression,
       boolean isCompaction) throws IOException {
     return MobUtils.createWriter(conf, region.getFilesystem(), family,
-      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, mobCacheConfig,
+      new Path(basePath, mobFileName.getFileName()), maxKeyCount, compression, cacheConf,
       cryptoContext, checksumType, bytesPerChecksum, blocksize, BloomType.NONE, isCompaction);
   }
 
@@ -291,7 +283,7 @@ public class HMobStore extends HStore {
   private void validateMobFile(Path path) throws IOException {
     HStoreFile storeFile = null;
     try {
-      storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
+      storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.cacheConf,
           BloomType.NONE, isPrimaryReplicaStore());
       storeFile.initReader();
     } catch (IOException e) {
@@ -398,11 +390,11 @@ public class HMobStore extends HStore {
       MobFile file = null;
       Path path = new Path(location, fileName);
       try {
-        file = mobCacheConfig.getMobFileCache().openFile(fs, path, mobCacheConfig);
+        file = mobFileCache.openFile(fs, path, cacheConf);
         return readPt != -1 ? file.readCell(search, cacheMobBlocks, readPt) : file.readCell(search,
           cacheMobBlocks);
       } catch (IOException e) {
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         throwable = e;
         if ((e instanceof FileNotFoundException) ||
             (e.getCause() instanceof FileNotFoundException)) {
@@ -414,16 +406,16 @@ public class HMobStore extends HStore {
           throw e;
         }
       } catch (NullPointerException e) { // HDFS 1.x - DFSInputStream.getBlockAt()
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         LOG.debug("Fail to read the cell", e);
         throwable = e;
       } catch (AssertionError e) { // assert in HDFS 1.x - DFSInputStream.getBlockAt()
-        mobCacheConfig.getMobFileCache().evictFile(fileName);
+        mobFileCache.evictFile(fileName);
         LOG.debug("Fail to read the cell", e);
         throwable = e;
       } finally {
         if (file != null) {
-          mobCacheConfig.getMobFileCache().closeFile(file);
+          mobFileCache.closeFile(file);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 4b6da53..21458c4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -131,11 +131,13 @@ import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
 import org.apache.hadoop.hbase.ipc.RpcCall;
 import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@@ -325,6 +327,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   final LongAdder compactionsQueued = new LongAdder();
   final LongAdder flushesQueued = new LongAdder();
 
+  private BlockCache blockCache;
+  private MobFileCache mobFileCache;
   private final WAL wal;
   private final HRegionFileSystem fs;
   protected final Configuration conf;
@@ -774,7 +778,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
       }
     }
+
     this.rsServices = rsServices;
+    if (this.rsServices != null) {
+      this.blockCache = rsServices.getBlockCache().orElse(null);
+      this.mobFileCache = rsServices.getMobFileCache().orElse(null);
+    }
+
     setHTableSpecificConf();
     this.scannerReadPoints = new ConcurrentHashMap<>();
 
@@ -1850,6 +1860,30 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return this.wal;
   }
 
+  public BlockCache getBlockCache() {
+    return this.blockCache;
+  }
+
+  /**
+   * Only used for unit test which doesn't start region server.
+   */
+  @VisibleForTesting
+  public void setBlockCache(BlockCache blockCache) {
+    this.blockCache = blockCache;
+  }
+
+  public MobFileCache getMobFileCache() {
+    return this.mobFileCache;
+  }
+
+  /**
+   * Only used for unit test which doesn't start region server.
+   */
+  @VisibleForTesting
+  public void setMobFileCache(MobFileCache mobFileCache) {
+    this.mobFileCache = mobFileCache;
+  }
+
   /**
    * @return split policy for this region.
    */
@@ -7062,7 +7096,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
     HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
     HRegion region = HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, null);
-    if (initialize) region.initialize(null);
+    if (initialize) {
+      region.initialize(null);
+    }
     return region;
   }
 
@@ -8130,7 +8166,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public static final long FIXED_OVERHEAD = ClassSize.align(
       ClassSize.OBJECT +
       ClassSize.ARRAY +
-      53 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
+      55 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
       (15 * Bytes.SIZEOF_LONG) +
       3 * Bytes.SIZEOF_BOOLEAN);