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/12/18 05:45:48 UTC

[1/3] hbase git commit: HBASE-21514 Refactor CacheConfig

Repository: hbase
Updated Branches:
  refs/heads/master 68b5df009 -> 1971d02e7


http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
index 444102d..2065c0c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java
@@ -28,12 +28,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
@@ -104,17 +106,15 @@ public class TestScannerSelectionUsingTTL {
   @Test
   public void testScannerSelection() throws IOException {
     Configuration conf = TEST_UTIL.getConfiguration();
-    CacheConfig.instantiateBlockCache(conf);
     conf.setBoolean("hbase.store.delete.expired.storefile", false);
-    HColumnDescriptor hcd =
-      new HColumnDescriptor(FAMILY_BYTES)
-          .setMaxVersions(Integer.MAX_VALUE)
-          .setTimeToLive(TTL_SECONDS);
-    HTableDescriptor htd = new HTableDescriptor(TABLE);
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(TABLE);
-    HRegion region = HBaseTestingUtility.createRegionAndWAL(info,
-      TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, htd);
+    LruBlockCache cache = (LruBlockCache) BlockCacheFactory.createBlockCache(conf);
+
+    TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE).setColumnFamily(
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES).setMaxVersions(Integer.MAX_VALUE)
+            .setTimeToLive(TTL_SECONDS).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TABLE).build();
+    HRegion region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(info.getEncodedName()), conf, td, cache);
 
     long ts = EnvironmentEdgeManager.currentTime();
     long version = 0; //make sure each new set of Put's have a new ts
@@ -136,10 +136,7 @@ public class TestScannerSelectionUsingTTL {
       version++;
     }
 
-    Scan scan = new Scan();
-    scan.setMaxVersions(Integer.MAX_VALUE);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    LruBlockCache cache = (LruBlockCache) cacheConf.getBlockCache();
+    Scan scan = new Scan().readVersions(Integer.MAX_VALUE);
     cache.clearCache();
     InternalScanner scanner = region.getScanner(scan);
     List<Cell> results = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 844b705..a930d7f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Random;
 import java.util.TreeMap;
 import java.util.concurrent.ConcurrentSkipListMap;
@@ -48,8 +49,10 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -708,4 +711,14 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
   public TableDescriptors getTableDescriptors() {
     return null;
   }
+
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.empty();
+  }
+
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.empty();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
index 75d9019..7ef8eca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNotCarryTable.java
@@ -79,7 +79,13 @@ public class TestMasterNotCarryTable {
   @Test
   public void testMasterBlockCache() {
     // no need to instantiate block cache.
-    assertNull(master.getCacheConfig().getBlockCache());
+    assertFalse(master.getBlockCache().isPresent());
+  }
+
+  @Test
+  public void testMasterMOBFileCache() {
+    // no need to instantiate mob file cache.
+    assertFalse(master.getMobFileCache().isPresent());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
index 325a580..b5bbd53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFileCache.java
@@ -33,6 +33,9 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.regionserver.HMobStore;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@@ -57,7 +60,6 @@ public class TestMobFileCache {
   private HBaseTestingUtility UTIL;
   private HRegion region;
   private Configuration conf;
-  private MobCacheConfig mobCacheConf;
   private MobFileCache mobFileCache;
   private Date currentDate = new Date();
   private static final String TEST_CACHE_SIZE = "2";
@@ -85,6 +87,7 @@ public class TestMobFileCache {
   public void setUp() throws Exception {
     UTIL = HBaseTestingUtility.createLocalHTU();
     conf = UTIL.getConfiguration();
+    conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
     HTableDescriptor htd = UTIL.createTableDescriptor("testMobFileCache");
     HColumnDescriptor hcd1 = new HColumnDescriptor(FAMILY1);
     hcd1.setMobEnabled(true);
@@ -98,7 +101,10 @@ public class TestMobFileCache {
     htd.addFamily(hcd1);
     htd.addFamily(hcd2);
     htd.addFamily(hcd3);
-    region = UTIL.createLocalHRegion(htd, null, null);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    mobFileCache = new MobFileCache(conf);
+    region = HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, mobFileCache);
   }
 
   @After
@@ -121,7 +127,6 @@ public class TestMobFileCache {
     HColumnDescriptor hcd = new HColumnDescriptor(family);
     hcd.setMaxVersions(4);
     hcd.setMobEnabled(true);
-    mobCacheConf = new MobCacheConfig(conf, hcd);
     return createMobStoreFile(hcd);
   }
 
@@ -158,17 +163,16 @@ public class TestMobFileCache {
   @Test
   public void testMobFileCache() throws Exception {
     FileSystem fs = FileSystem.get(conf);
-    conf.set(MobConstants.MOB_FILE_CACHE_SIZE_KEY, TEST_CACHE_SIZE);
-    mobFileCache = new MobFileCache(conf);
     Path file1Path = createMobStoreFile(FAMILY1);
     Path file2Path = createMobStoreFile(FAMILY2);
     Path file3Path = createMobStoreFile(FAMILY3);
 
+    CacheConfig cacheConf = new CacheConfig(conf);
     // Before open one file by the MobFileCache
     assertEquals(EXPECTED_CACHE_SIZE_ZERO, mobFileCache.getCacheSize());
     // Open one file by the MobFileCache
     CachedMobFile cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
-        fs, file1Path, mobCacheConf);
+        fs, file1Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
     assertNotNull(cachedMobFile1);
     assertEquals(EXPECTED_REFERENCE_TWO, cachedMobFile1.getReferenceCount());
@@ -188,13 +192,13 @@ public class TestMobFileCache {
 
     // Reopen three cached file
     cachedMobFile1 = (CachedMobFile) mobFileCache.openFile(
-        fs, file1Path, mobCacheConf);
+        fs, file1Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_ONE, mobFileCache.getCacheSize());
     CachedMobFile cachedMobFile2 = (CachedMobFile) mobFileCache.openFile(
-        fs, file2Path, mobCacheConf);
+        fs, file2Path, cacheConf);
     assertEquals(EXPECTED_CACHE_SIZE_TWO, mobFileCache.getCacheSize());
     CachedMobFile cachedMobFile3 = (CachedMobFile) mobFileCache.openFile(
-        fs, file3Path, mobCacheConf);
+        fs, file3Path, cacheConf);
     // Before the evict
     // Evict the cache, should close the first file 1
     assertEquals(EXPECTED_CACHE_SIZE_THREE, mobFileCache.getCacheSize());

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
index 59f6678..0fb183e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java
@@ -740,12 +740,7 @@ public class DataBlockEncodingTool {
         "Excluding the first " + benchmarkNOmit + " times from statistics.");
 
     final Configuration conf = HBaseConfiguration.create();
-    try {
-      testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark,
-          doVerify);
-    } finally {
-      (new CacheConfig(conf)).getBlockCache().shutdown();
-    }
+    testCodecs(conf, kvLimit, pathName, compressionName, doBenchmark, doVerify);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
index 57948b6..dcb6c3f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java
@@ -180,6 +180,6 @@ public class EncodedSeekPerformanceTest {
   }
 
   private void clearBlockCache() {
-    ((LruBlockCache) cacheConf.getBlockCache()).clearCache();
+    ((LruBlockCache) cacheConf.getBlockCache().get()).clearCache();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 16381d2..b449f37 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -114,14 +115,17 @@ public class TestAtomicOperation {
   @After
   public void teardown() throws IOException {
     if (region != null) {
-      BlockCache bc = region.getStores().get(0).getCacheConfig().getBlockCache();
+      CacheConfig cacheConfig = region.getStores().get(0).getCacheConfig();
       region.close();
       WAL wal = region.getWAL();
-      if (wal != null) wal.close();
-      if (bc != null) bc.shutdown();
+      if (wal != null) {
+        wal.close();
+      }
+      cacheConfig.getBlockCache().ifPresent(BlockCache::shutdown);
       region = null;
     }
   }
+
   //////////////////////////////////////////////////////////////////////////////
   // New tests that doesn't spin up a mini cluster but rather just test the
   // individual code pieces in the HRegion.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
index 2cf3f8c..4425762 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksRead.java
@@ -30,24 +30,28 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.junit.*;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
@@ -61,12 +65,12 @@ public class TestBlocksRead  {
       HBaseClassTestRule.forClass(TestBlocksRead.class);
 
   private static final Logger LOG = LoggerFactory.getLogger(TestBlocksRead.class);
-  @Rule public TestName testName = new TestName();
+  @Rule
+  public TestName testName = new TestName();
 
   static final BloomType[] BLOOM_TYPE = new BloomType[] { BloomType.ROWCOL,
       BloomType.ROW, BloomType.NONE };
 
-  private static BlockCache blockCache;
   HRegion region = null;
   private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private final String DIR = TEST_UTIL.getDataTestDir("TestBlocksRead").toString();
@@ -76,7 +80,6 @@ public class TestBlocksRead  {
   public static void setUp() throws Exception {
     // disable compactions in this test.
     TEST_UTIL.getConfiguration().setInt("hbase.hstore.compactionThreshold", 10000);
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
   }
 
   @AfterClass
@@ -93,23 +96,31 @@ public class TestBlocksRead  {
    * @throws IOException
    * @return created and initialized region.
    */
-  private HRegion initHRegion(byte[] tableName, String callingMethod,
-      Configuration conf, String family) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
-    HColumnDescriptor familyDesc;
+  private HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+      String family) throws IOException {
+    return initHRegion(tableName, callingMethod, conf, family, null);
+  }
+
+  /**
+   * Callers must afterward call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
+   */
+  private HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
+      String family, BlockCache blockCache) throws IOException {
+    TableDescriptorBuilder builder =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName));
     for (int i = 0; i < BLOOM_TYPE.length; i++) {
       BloomType bloomType = BLOOM_TYPE[i];
-      familyDesc = new HColumnDescriptor(family + "_" + bloomType)
-          .setBlocksize(1)
-          .setBloomFilterType(BLOOM_TYPE[i]);
-      htd.addFamily(familyDesc);
+      builder.setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(family + "_" + bloomType))
+              .setBlocksize(1).setBloomFilterType(bloomType).build());
     }
-
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
     Path path = new Path(DIR + callingMethod);
-    HRegion r = HBaseTestingUtility.createRegionAndWAL(info, path, conf, htd);
-    blockCache = new CacheConfig(conf).getBlockCache();
-    return r;
+    if (blockCache != null) {
+      return HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build(), blockCache);
+    } else {
+      return HBaseTestingUtility.createRegionAndWAL(info, path, conf, builder.build());
+    }
   }
 
   private void putData(String family, String row, String col, long version)
@@ -204,10 +215,6 @@ public class TestBlocksRead  {
       return HFile.DATABLOCK_READ_COUNT.sum();
   }
 
-  private static long getBlkCount() {
-    return blockCache.getBlockCount();
-  }
-
   /**
    * Test # of blocks read for some simple seek cases.
    *
@@ -381,7 +388,8 @@ public class TestBlocksRead  {
     byte [] TABLE = Bytes.toBytes("testBlocksReadWhenCachingDisabled");
     String FAMILY = "cf1";
 
-    this.region = initHRegion(TABLE, testName.getMethodName(), conf, FAMILY);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(conf);
+    this.region = initHRegion(TABLE, testName.getMethodName(), conf, FAMILY, blockCache);
 
     try {
       putData(FAMILY, "row", "col1", 1);
@@ -390,7 +398,7 @@ public class TestBlocksRead  {
 
       // Execute a scan with caching turned off
       // Expected blocks stored: 0
-      long blocksStart = getBlkCount();
+      long blocksStart = blockCache.getBlockCount();
       Scan scan = new Scan();
       scan.setCacheBlocks(false);
       RegionScanner rs = region.getScanner(scan);
@@ -398,7 +406,7 @@ public class TestBlocksRead  {
       rs.next(result);
       assertEquals(2 * BLOOM_TYPE.length, result.size());
       rs.close();
-      long blocksEnd = getBlkCount();
+      long blocksEnd = blockCache.getBlockCount();
 
       assertEquals(blocksStart, blocksEnd);
 
@@ -411,7 +419,7 @@ public class TestBlocksRead  {
       rs.next(result);
       assertEquals(2 * BLOOM_TYPE.length, result.size());
       rs.close();
-      blocksEnd = getBlkCount();
+      blocksEnd = blockCache.getBlockCount();
 
       assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
     } finally {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
index 0ba4e97..c7bc168 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestBlocksScanned.java
@@ -17,25 +17,33 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import static org.apache.hadoop.hbase.HBaseTestCase.addContent;
+import static org.junit.Assert.assertEquals;
+
 import java.util.ArrayList;
 import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestCase;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -43,7 +51,7 @@ import org.junit.experimental.categories.Category;
 
 @SuppressWarnings("deprecation")
 @Category({RegionServerTests.class, SmallTests.class})
-public class TestBlocksScanned extends HBaseTestCase {
+public class TestBlocksScanned {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
@@ -56,13 +64,14 @@ public class TestBlocksScanned extends HBaseTestCase {
   private static int BLOCK_SIZE = 70;
 
   private static HBaseTestingUtility TEST_UTIL = null;
+  private Configuration conf;
+  private Path testDir;
 
-  @Override
   @Before
   public void setUp() throws Exception {
-    super.setUp();
     TEST_UTIL = new HBaseTestingUtility();
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
+    conf = TEST_UTIL.getConfiguration();
+    testDir = TEST_UTIL.getDataTestDir("TestBlocksScanned");
   }
 
   @Test
@@ -96,12 +105,16 @@ public class TestBlocksScanned extends HBaseTestCase {
     _testBlocksScanned(table);
   }
 
-  private void _testBlocksScanned(HTableDescriptor table) throws Exception {
-    HRegion r = createNewHRegion(table, START_KEY, END_KEY, TEST_UTIL.getConfiguration());
+  private void _testBlocksScanned(TableDescriptor td) throws Exception {
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(conf);
+    RegionInfo regionInfo =
+        RegionInfoBuilder.newBuilder(td.getTableName()).setStartKey(START_KEY).setEndKey(END_KEY)
+            .build();
+    HRegion r = HBaseTestingUtility.createRegionAndWAL(regionInfo, testDir, conf, td, blockCache);
     addContent(r, FAMILY, COL);
     r.flush(true);
 
-    CacheStats stats = new CacheConfig(TEST_UTIL.getConfiguration()).getBlockCache().getStats();
+    CacheStats stats = blockCache.getStats();
     long before = stats.getHitCount() + stats.getMissCount();
     // Do simple test of getting one row only first.
     Scan scan = new Scan().withStartRow(Bytes.toBytes("aaa")).withStopRow(Bytes.toBytes("aaz"))
@@ -119,11 +132,12 @@ public class TestBlocksScanned extends HBaseTestCase {
 
     int kvPerBlock = (int) Math.ceil(BLOCK_SIZE /
         (double) KeyValueUtil.ensureKeyValue(results.get(0)).getLength());
-    Assert.assertEquals(2, kvPerBlock);
+    assertEquals(2, kvPerBlock);
 
     long expectDataBlockRead = (long) Math.ceil(expectResultSize / (double) kvPerBlock);
     long expectIndexBlockRead = expectDataBlockRead;
 
-    assertEquals(expectIndexBlockRead+expectDataBlockRead, stats.getHitCount() + stats.getMissCount() - before);
+    assertEquals(expectIndexBlockRead + expectDataBlockRead,
+        stats.getHitCount() + stats.getMissCount() - before);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
index dc51dae..1248b21 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
 import org.apache.hadoop.hbase.io.hfile.BlockType;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@@ -161,8 +162,6 @@ public class TestCacheOnWriteInSchema {
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
     conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, false);
     conf.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, false);
-    CacheConfig.instantiateBlockCache(conf);
-
     fs = HFileSystem.get(conf);
 
     // Create the schema
@@ -182,6 +181,7 @@ public class TestCacheOnWriteInSchema {
     walFactory = new WALFactory(conf, id);
 
     region = TEST_UTIL.createLocalHRegion(info, htd, walFactory.getWAL(info));
+    region.setBlockCache(BlockCacheFactory.createBlockCache(conf));
     store = new HStore(region, hcd, conf);
   }
 
@@ -224,7 +224,7 @@ public class TestCacheOnWriteInSchema {
 
   private void readStoreFile(Path path) throws IOException {
     CacheConfig cacheConf = store.getCacheConfig();
-    BlockCache cache = cacheConf.getBlockCache();
+    BlockCache cache = cacheConf.getBlockCache().get();
     HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
     sf.initReader();
     HFile.Reader reader = sf.getReader().getHFileReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
index 47fc2d7..3c8e8be 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestClearRegionBlockCache.java
@@ -100,8 +100,8 @@ public class TestClearRegionBlockCache {
 
   @Test
   public void testClearBlockCache() throws Exception {
-    BlockCache blockCache1 = rs1.getCacheConfig().getBlockCache();
-    BlockCache blockCache2 = rs2.getCacheConfig().getBlockCache();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
 
     long initialBlockCount1 = blockCache1.getBlockCount();
     long initialBlockCount2 = blockCache2.getBlockCount();
@@ -125,23 +125,24 @@ public class TestClearRegionBlockCache {
   public void testClearBlockCacheFromAdmin() throws Exception {
     Admin admin = HTU.getAdmin();
 
-    // All RS run in a same process, so the block cache is same for rs1 and rs2
-    BlockCache blockCache = rs1.getCacheConfig().getBlockCache();
-    long initialBlockCount = blockCache.getBlockCount();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
+    long initialBlockCount1 = blockCache1.getBlockCount();
+    long initialBlockCount2 = blockCache2.getBlockCount();
 
     // scan will cause blocks to be added in BlockCache
     scanAllRegionsForRS(rs1);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
+    assertEquals(blockCache1.getBlockCount() - initialBlockCount1,
+        HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
     scanAllRegionsForRS(rs2);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-          + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(blockCache2.getBlockCount() - initialBlockCount2,
+        HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
 
     CacheEvictionStats stats = admin.clearBlockCache(TABLE_NAME);
     assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
         + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
-    assertEquals(initialBlockCount, blockCache.getBlockCount());
+    assertEquals(initialBlockCount1, blockCache1.getBlockCount());
+    assertEquals(initialBlockCount2, blockCache2.getBlockCount());
   }
 
   @Test
@@ -149,23 +150,24 @@ public class TestClearRegionBlockCache {
     AsyncAdmin admin =
         ConnectionFactory.createAsyncConnection(HTU.getConfiguration()).get().getAdmin();
 
-    // All RS run in a same process, so the block cache is same for rs1 and rs2
-    BlockCache blockCache = rs1.getCacheConfig().getBlockCache();
-    long initialBlockCount = blockCache.getBlockCount();
+    BlockCache blockCache1 = rs1.getBlockCache().get();
+    BlockCache blockCache2 = rs2.getBlockCache().get();
+    long initialBlockCount1 = blockCache1.getBlockCount();
+    long initialBlockCount2 = blockCache2.getBlockCount();
 
     // scan will cause blocks to be added in BlockCache
     scanAllRegionsForRS(rs1);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
+    assertEquals(blockCache1.getBlockCount() - initialBlockCount1,
+        HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY));
     scanAllRegionsForRS(rs2);
-    assertEquals(blockCache.getBlockCount() - initialBlockCount,
-      HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-          + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(blockCache2.getBlockCount() - initialBlockCount2,
+        HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
 
     CacheEvictionStats stats = admin.clearBlockCache(TABLE_NAME).get();
-    assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY)
-        + HTU.getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
-    assertEquals(initialBlockCount, blockCache.getBlockCount());
+    assertEquals(stats.getEvictedBlocks(), HTU.getNumHFilesForRS(rs1, TABLE_NAME, FAMILY) + HTU
+        .getNumHFilesForRS(rs2, TABLE_NAME, FAMILY));
+    assertEquals(initialBlockCount1, blockCache1.getBlockCount());
+    assertEquals(initialBlockCount2, blockCache2.getBlockCount());
   }
 
   private void scanAllRegionsForRS(HRegionServer rs) throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
index 4c2d645..3d22b73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
@@ -41,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
 import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
@@ -122,7 +122,6 @@ public class TestCompoundBloomFilter {
   private static Configuration conf;
   private static CacheConfig cacheConf;
   private FileSystem fs;
-  private BlockCache blockCache;
 
   /** A message of the form "in test#&lt;number>:" to include in logging. */
   private String testIdMsg;
@@ -130,6 +129,8 @@ public class TestCompoundBloomFilter {
   private static final int GENERATION_SEED = 2319;
   private static final int EVALUATION_SEED = 135;
 
+  private BlockCache blockCache;
+
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
@@ -138,11 +139,8 @@ public class TestCompoundBloomFilter {
     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
 
     fs = FileSystem.get(conf);
-
-    CacheConfig.instantiateBlockCache(conf);
-    cacheConf = new CacheConfig(conf);
-    blockCache = cacheConf.getBlockCache();
-    assertNotNull(blockCache);
+    blockCache = BlockCacheFactory.createBlockCache(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
   }
 
   private List<KeyValue> createSortedKeyValues(Random rand, int n) {
@@ -305,7 +303,7 @@ public class TestCompoundBloomFilter {
     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
         BLOOM_BLOCK_SIZES[t]);
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
     HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build();
     StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs)
             .withOutputDir(TEST_UTIL.getDataTestDir())
@@ -373,7 +371,5 @@ public class TestCompoundBloomFilter {
       rowColKV.getRowLength()));
     assertEquals(0, rowKV.getQualifierLength());
   }
-
-
 }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 9c5a667..2001602 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -32,6 +32,7 @@ import java.util.NavigableSet;
 import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListSet;
 import javax.crypto.spec.SecretKeySpec;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -42,20 +43,24 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
 import org.apache.hadoop.hbase.io.crypto.aes.AES;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
@@ -90,7 +95,6 @@ public class TestHMobStore {
 
   private HMobStore store;
   private HRegion region;
-  private HColumnDescriptor hcd;
   private FileSystem fs;
   private byte [] table = Bytes.toBytes("table");
   private byte [] family = Bytes.toBytes("family");
@@ -131,49 +135,44 @@ public class TestHMobStore {
       byte [] next = iter.next();
       expected.add(new KeyValue(row, family, next, 1, value));
       get.addColumn(family, next);
-      get.setMaxVersions(); // all versions.
+      get.readAllVersions();
     }
   }
 
-  private void init(String methodName, Configuration conf, boolean testStore)
-  throws IOException {
-    hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(3L);
-    hcd.setMaxVersions(4);
-    init(methodName, conf, hcd, testStore);
+  private void init(String methodName, Configuration conf, boolean testStore) throws IOException {
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(3L)
+            .setMaxVersions(4).build();
+    init(methodName, conf, cfd, testStore);
   }
 
-  private void init(String methodName, Configuration conf,
-      HColumnDescriptor hcd, boolean testStore) throws IOException {
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
-    init(methodName, conf, htd, hcd, testStore);
-  }
+  private void init(String methodName, Configuration conf, ColumnFamilyDescriptor cfd,
+      boolean testStore) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).setColumnFamily(cfd).build();
 
-  private void init(String methodName, Configuration conf, HTableDescriptor htd,
-      HColumnDescriptor hcd, boolean testStore) throws IOException {
     //Setting up tje Region and Store
-    Path basedir = new Path(DIR+methodName);
-    Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
+    Path basedir = new Path(DIR + methodName);
+    Path tableDir = FSUtils.getTableDir(basedir, td.getTableName());
     String logName = "logs";
     Path logdir = new Path(basedir, logName);
     FileSystem fs = FileSystem.get(conf);
     fs.delete(logdir, true);
 
-    htd.addFamily(hcd);
-    HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
+    RegionInfo info = RegionInfoBuilder.newBuilder(td.getTableName()).build();
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     final Configuration walConf = new Configuration(conf);
     FSUtils.setRootDir(walConf, basedir);
     final WALFactory wals = new WALFactory(walConf, methodName);
-    region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, htd, null);
-    store = new HMobStore(region, hcd, conf);
-    if(testStore) {
-      init(conf, hcd);
+    region = new HRegion(tableDir, wals.getWAL(info), fs, conf, info, td, null);
+    region.setMobFileCache(new MobFileCache(conf));
+    store = new HMobStore(region, cfd, conf);
+    if (testStore) {
+      init(conf, cfd);
     }
   }
 
-  private void init(Configuration conf, HColumnDescriptor hcd)
+  private void init(Configuration conf, ColumnFamilyDescriptor cfd)
       throws IOException {
     Path basedir = FSUtils.getRootDir(conf);
     fs = FileSystem.get(conf);
@@ -187,7 +186,7 @@ public class TestHMobStore {
     KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
     int maxKeyCount = keys.length;
     StoreFileWriter mobWriter = store.createWriterInTmp(currentDate, maxKeyCount,
-        hcd.getCompactionCompressionType(), region.getRegionInfo().getStartKey(), false);
+        cfd.getCompactionCompressionType(), region.getRegionInfo().getStartKey(), false);
     mobFilePath = mobWriter.getPath();
 
     mobWriter.append(key1);
@@ -382,15 +381,11 @@ public class TestHMobStore {
    */
   @Test
   public void testMobCellSizeThreshold() throws IOException {
-
     final Configuration conf = HBaseConfiguration.create();
-
-    HColumnDescriptor hcd;
-    hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(100);
-    hcd.setMaxVersions(4);
-    init(name.getMethodName(), conf, hcd, false);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(100)
+            .setMaxVersions(4).build();
+    init(name.getMethodName(), conf, cfd, false);
 
     //Put data in memstore
     this.store.add(new KeyValue(row, family, qf1, 1, value), null);
@@ -503,15 +498,12 @@ public class TestHMobStore {
     String algorithm = conf.get(HConstants.CRYPTO_KEY_ALGORITHM_CONF_KEY, HConstants.CIPHER_AES);
     Key cfKey = new SecretKeySpec(keyBytes, algorithm);
 
-    HColumnDescriptor hcd = new HColumnDescriptor(family);
-    hcd.setMobEnabled(true);
-    hcd.setMobThreshold(100);
-    hcd.setMaxVersions(4);
-    hcd.setEncryptionType(algorithm);
-    hcd.setEncryptionKey(EncryptionUtil.wrapKey(conf,
-      conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, User.getCurrent().getShortName()),cfKey));
-
-    init(name.getMethodName(), conf, hcd, false);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(family).setMobEnabled(true).setMobThreshold(100)
+            .setMaxVersions(4).setEncryptionType(algorithm).setEncryptionKey(EncryptionUtil
+            .wrapKey(conf, conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY,
+                User.getCurrent().getShortName()), cfKey)).build();
+    init(name.getMethodName(), conf, cfd, false);
 
     this.store.add(new KeyValue(row, family, qf1, 1, value), null);
     this.store.add(new KeyValue(row, family, qf2, 1, value), null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
index 30ee3b2..ac63974 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.HFileLink;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -91,7 +92,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestHStoreFile.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private CacheConfig cacheConf =  new CacheConfig(TEST_UTIL.getConfiguration());
+  private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
   private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString();
   private static final ChecksumType CKTYPE = ChecksumType.CRC32C;
   private static final int CKBYTES = 512;
@@ -934,8 +935,7 @@ public class TestHStoreFile extends HBaseTestCase {
     Path baseDir = new Path(new Path(testDir, "7e0102"),"twoCOWEOC");
 
     // Grab the block cache and get the initial hit/miss counts
-    CacheConfig.instantiateBlockCache(conf);
-    BlockCache bc = new CacheConfig(conf).getBlockCache();
+    BlockCache bc = BlockCacheFactory.createBlockCache(conf);
     assertNotNull(bc);
     CacheStats cs = bc.getStats();
     long startHit = cs.getHitCount();
@@ -944,7 +944,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's write a StoreFile with three blocks, with cache on write off
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
-    CacheConfig cacheConf = new CacheConfig(conf);
+    CacheConfig cacheConf = new CacheConfig(conf, bc);
     Path pathCowOff = new Path(baseDir, "123456789");
     StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
     HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
@@ -967,7 +967,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Now write a StoreFile with three blocks, with cache on write on
     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     Path pathCowOn = new Path(baseDir, "123456788");
     writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
     hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
@@ -1025,7 +1025,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's close the first file with evict on close turned on
     conf.setBoolean("hbase.rs.evictblocksonclose", true);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
     hsf.initReader();
     reader = hsf.getReader();
@@ -1039,7 +1039,7 @@ public class TestHStoreFile extends HBaseTestCase {
 
     // Let's close the second file with evict on close turned off
     conf.setBoolean("hbase.rs.evictblocksonclose", false);
-    cacheConf = new CacheConfig(conf);
+    cacheConf = new CacheConfig(conf, bc);
     hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
     hsf.initReader();
     reader = hsf.getReader();

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
index cc9e385..cccc970 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -49,6 +50,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
@@ -58,14 +61,13 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hadoop.hbase.mob.MobConstants;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -101,16 +103,6 @@ public class TestMobStoreCompaction {
   private final byte[] STARTROW = Bytes.toBytes(START_KEY);
   private int compactionThreshold;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws Exception {
-    UTIL.startMiniCluster(1);
-  }
-
-  @AfterClass
-  public static void tearDownAfterClass() throws Exception {
-    UTIL.shutdownMiniCluster();
-  }
-
   private void init(Configuration conf, long mobThreshold) throws Exception {
     this.conf = conf;
     this.mobCellThreshold = mobThreshold;
@@ -124,7 +116,9 @@ public class TestMobStoreCompaction {
     hcd.setMaxVersions(1);
     htd.modifyFamily(hcd);
 
-    region = UTIL.createLocalHRegion(htd, null, null);
+    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    region = HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, UTIL.getDataTestDir(), conf, htd, new MobFileCache(conf));
     fs = FileSystem.get(conf);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
index bb97c9c..ab591ad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiColumnScanner.java
@@ -36,15 +36,16 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueTestUtil;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.util.BloomFilterUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
@@ -139,12 +140,10 @@ public abstract class TestMultiColumnScanner {
     TEST_UTIL.getConfiguration().setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, 10);
     TEST_UTIL.getConfiguration().set(BloomFilterUtil.DELIMITER_KEY, "#");
     HRegion region = TEST_UTIL.createTestRegion(TABLE_NAME,
-        new HColumnDescriptor(FAMILY)
-            .setCompressionType(comprAlgo)
-            .setBloomFilterType(bloomType)
-            .setMaxVersions(MAX_VERSIONS)
-            .setDataBlockEncoding(dataBlockEncoding)
-    );
+        ColumnFamilyDescriptorBuilder.newBuilder(FAMILY_BYTES).setCompressionType(comprAlgo)
+            .setBloomFilterType(bloomType).setMaxVersions(MAX_VERSIONS)
+            .setDataBlockEncoding(dataBlockEncoding).build(),
+        BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration()));
     List<String> rows = sequentialStrings("row", NUM_ROWS);
     List<String> qualifiers = sequentialStrings("qual", NUM_COLUMNS);
     List<KeyValue> kvs = new ArrayList<>();
@@ -202,7 +201,7 @@ public abstract class TestMultiColumnScanner {
     for (int maxVersions = 1; maxVersions <= TIMESTAMPS.length; ++maxVersions) {
       for (int columnBitMask = 1; columnBitMask <= MAX_COLUMN_BIT_MASK; ++columnBitMask) {
         Scan scan = new Scan();
-        scan.setMaxVersions(maxVersions);
+        scan.readVersions(maxVersions);
         Set<String> qualSet = new TreeSet<>();
         {
           int columnMaskTmp = columnBitMask;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
index 17b01b5..fd3a56d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRSStatusServlet.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.util.List;
+import java.util.Optional;
+
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +30,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServer;
 import org.apache.hadoop.hbase.ipc.MetricsHBaseServerWrapperStub;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -102,11 +103,9 @@ public class TestRSStatusServlet {
     Mockito.doReturn("fakequorum").when(zkw).getQuorum();
     Mockito.doReturn(zkw).when(rs).getZooKeeper();
 
-    // Fake CacheConfig
+    // Fake BlockCache
     LOG.warn("The " + HConstants.HFILE_BLOCK_CACHE_SIZE_KEY + " is set to 0");
-    CacheConfig cacheConf = Mockito.mock(CacheConfig.class);
-    Mockito.doReturn(null).when(cacheConf).getBlockCache();
-    Mockito.doReturn(cacheConf).when(rs).getCacheConfig();
+    Mockito.doReturn(Optional.empty()).when(rs).getBlockCache();
 
     // Fake MasterAddressTracker
     MasterAddressTracker mat = Mockito.mock(MasterAddressTracker.class);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
index 543126e..f8bd48c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRecoveredEdits.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.List;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,16 +32,19 @@ import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MemoryCompactionPolicy;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -49,6 +53,7 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALKey;
 import org.apache.hadoop.hbase.wal.WALSplitter;
+import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -69,8 +74,16 @@ public class TestRecoveredEdits {
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static final Logger LOG = LoggerFactory.getLogger(TestRecoveredEdits.class);
+
+  private static BlockCache blockCache;
+
   @Rule public TestName testName = new TestName();
 
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    blockCache = BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration());
+  }
+
   /**
    * HBASE-12782 ITBLL fails for me if generator does anything but 5M per maptask.
    * Create a region. Close it. Then copy into place a file to replay, one that is bigger than
@@ -81,7 +94,6 @@ public class TestRecoveredEdits {
   @Test
   public void testReplayWorksThoughLotsOfFlushing() throws
       IOException {
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
     for(MemoryCompactionPolicy policy : MemoryCompactionPolicy.values()) {
       testReplayWorksWithMemoryCompactionPolicy(policy);
     }
@@ -96,18 +108,22 @@ public class TestRecoveredEdits {
     // The file of recovered edits has a column family of 'meta'. Also has an encoded regionname
     // of 4823016d8fca70b25503ee07f4c6d79f which needs to match on replay.
     final String encodedRegionName = "4823016d8fca70b25503ee07f4c6d79f";
-    HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(testName.getMethodName()));
     final String columnFamily = "meta";
     byte [][] columnFamilyAsByteArray = new byte [][] {Bytes.toBytes(columnFamily)};
-    htd.addFamily(new HColumnDescriptor(columnFamily));
-    HRegionInfo hri = new HRegionInfo(htd.getTableName()) {
+    TableDescriptor tableDescriptor =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(testName.getMethodName()))
+            .setColumnFamily(
+                ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(columnFamily)).build())
+            .build();
+    RegionInfo hri = new HRegionInfo(tableDescriptor.getTableName()) {
       @Override
       public synchronized String getEncodedName() {
         return encodedRegionName;
       }
 
       // Cache the name because lots of lookups.
-      private byte [] encodedRegionNameAsBytes = null;
+      private byte[] encodedRegionNameAsBytes = null;
+
       @Override
       public synchronized byte[] getEncodedNameAsBytes() {
         if (encodedRegionNameAsBytes == null) {
@@ -117,16 +133,16 @@ public class TestRecoveredEdits {
       }
     };
     Path hbaseRootDir = TEST_UTIL.getDataTestDir();
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
     FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration());
-    Path tableDir = FSUtils.getTableDir(hbaseRootDir, htd.getTableName());
+    Path tableDir = FSUtils.getTableDir(hbaseRootDir, tableDescriptor.getTableName());
     HRegionFileSystem hrfs =
         new HRegionFileSystem(TEST_UTIL.getConfiguration(), fs, tableDir, hri);
     if (fs.exists(hrfs.getRegionDir())) {
       LOG.info("Region directory already exists. Deleting.");
       fs.delete(hrfs.getRegionDir(), true);
     }
-    HRegion region = HRegion.createHRegion(hri, hbaseRootDir, conf, htd, null);
+    HRegion region = HBaseTestingUtility
+        .createRegionAndWAL(hri, hbaseRootDir, conf, tableDescriptor, blockCache);
     assertEquals(encodedRegionName, region.getRegionInfo().getEncodedName());
     List<String> storeFiles = region.getStoreFileList(columnFamilyAsByteArray);
     // There should be no store files.

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
index 61de21f..647f450 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRowPrefixBloomFilter.java
@@ -68,7 +68,7 @@ public class TestRowPrefixBloomFilter {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestRowPrefixBloomFilter.class);
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private CacheConfig cacheConf =  new CacheConfig(TEST_UTIL.getConfiguration());
+  private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
   private static final ChecksumType CKTYPE = ChecksumType.CRC32C;
   private static final int CKBYTES = 512;
   private boolean localfs = false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
index 75ebfd3..eb25806 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSecureBulkLoadManager.java
@@ -214,7 +214,7 @@ public class TestSecureBulkLoadManager {
     ColumnFamilyDescriptor family = desc.getColumnFamily(FAMILY);
     Compression.Algorithm compression = HFile.DEFAULT_COMPRESSION_ALGORITHM;
 
-    CacheConfig writerCacheConf = new CacheConfig(conf, family);
+    CacheConfig writerCacheConf = new CacheConfig(conf, family, null);
     writerCacheConf.setCacheDataOnWrite(false);
     HFileContext hFileContext = new HFileContextBuilder()
         .withIncludesMvcc(false)


[2/3] hbase git commit: HBASE-21514 Refactor CacheConfig

Posted by zg...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/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 6242d36..13f277b 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
@@ -36,6 +36,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Objects;
+import java.util.Optional;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.Timer;
@@ -98,7 +99,7 @@ import org.apache.hadoop.hbase.executor.ExecutorType;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.http.InfoServer;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
@@ -114,7 +115,7 @@ import org.apache.hadoop.hbase.log.HBaseMarkers;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.RegionState.State;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
 import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
 import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
@@ -410,10 +411,10 @@ public class HRegionServer extends HasThread implements
 
   private final RegionServerAccounting regionServerAccounting;
 
-  // Cache configuration and block cache reference
-  protected CacheConfig cacheConfig;
-  // Cache configuration for mob
-  final MobCacheConfig mobCacheConfig;
+  // Block cache
+  private BlockCache blockCache;
+  // The cache for mob files
+  private MobFileCache mobFileCache;
 
   /** The health check chore. */
   private HealthCheckChore healthCheckChore;
@@ -591,12 +592,12 @@ public class HRegionServer extends HasThread implements
 
       boolean isMasterNotCarryTable =
           this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf);
-      // no need to instantiate global block cache when master not carry table
+
+      // no need to instantiate block cache and mob file cache when master not carry table
       if (!isMasterNotCarryTable) {
-        CacheConfig.instantiateBlockCache(conf);
+        blockCache = BlockCacheFactory.createBlockCache(conf);
+        mobFileCache = new MobFileCache(conf);
       }
-      cacheConfig = new CacheConfig(conf);
-      mobCacheConfig = new MobCacheConfig(conf);
 
       uncaughtExceptionHandler = new UncaughtExceptionHandler() {
         @Override
@@ -1062,10 +1063,12 @@ public class HRegionServer extends HasThread implements
       }
     }
     // Send cache a shutdown.
-    if (cacheConfig != null && cacheConfig.isBlockCacheEnabled()) {
-      cacheConfig.getBlockCache().shutdown();
+    if (blockCache != null) {
+      blockCache.shutdown();
+    }
+    if (mobFileCache != null) {
+      mobFileCache.shutdown();
     }
-    mobCacheConfig.getMobFileCache().shutdown();
 
     if (movedRegionsCleaner != null) {
       movedRegionsCleaner.stop("Region Server stopping");
@@ -1607,9 +1610,9 @@ public class HRegionServer extends HasThread implements
   }
 
   private void startHeapMemoryManager() {
-    this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this,
-        this.regionServerAccounting);
-    if (this.hMemManager != null) {
+    if (this.blockCache != null) {
+      this.hMemManager =
+          new HeapMemoryManager(this.blockCache, this.cacheFlusher, this, regionServerAccounting);
       this.hMemManager.start(getChoreService());
     }
   }
@@ -3614,10 +3617,23 @@ public class HRegionServer extends HasThread implements
   }
 
   /**
-   * @return The cache config instance used by the regionserver.
+   * May be null if this is a master which not carry table.
+   *
+   * @return The block cache instance used by the regionserver.
+   */
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.ofNullable(this.blockCache);
+  }
+
+  /**
+   * May be null if this is a master which not carry table.
+   *
+   * @return The cache for mob files used by the regionserver.
    */
-  public CacheConfig getCacheConfig() {
-    return this.cacheConfig;
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.ofNullable(this.mobFileCache);
   }
 
   /**
@@ -3646,7 +3662,6 @@ public class HRegionServer extends HasThread implements
   }
 
   public CacheEvictionStats clearRegionBlockCache(Region region) {
-    BlockCache blockCache = this.getCacheConfig().getBlockCache();
     long evictedBlocks = 0;
 
     for(Store store : region.getStores()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index 032dc5f..b3e5b97 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -377,7 +377,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
    * @param family The current column family.
    */
   protected void createCacheConf(final ColumnFamilyDescriptor family) {
-    this.cacheConf = new CacheConfig(conf, family);
+    this.cacheConf = new CacheConfig(conf, family, region.getBlockCache());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
index c32fce2..a96417d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java
@@ -30,13 +30,14 @@ import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ScheduledChore;
 import org.apache.hadoop.hbase.Server;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
 import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
 import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 
@@ -105,20 +106,11 @@ public class HeapMemoryManager {
 
   private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<>();
 
-  public static HeapMemoryManager create(Configuration conf, FlushRequester memStoreFlusher,
-      Server server, RegionServerAccounting regionServerAccounting) {
-    ResizableBlockCache lruCache = CacheConfig.getOnHeapCache(conf);
-    if (lruCache != null) {
-      return new HeapMemoryManager(lruCache, memStoreFlusher, server, regionServerAccounting);
-    }
-    return null;
-  }
-
   @VisibleForTesting
-  HeapMemoryManager(ResizableBlockCache blockCache, FlushRequester memStoreFlusher,
+  HeapMemoryManager(BlockCache blockCache, FlushRequester memStoreFlusher,
                 Server server, RegionServerAccounting regionServerAccounting) {
     Configuration conf = server.getConfiguration();
-    this.blockCache = blockCache;
+    this.blockCache = toResizableBlockCache(blockCache);
     this.memStoreFlusher = memStoreFlusher;
     this.server = server;
     this.regionServerAccounting = regionServerAccounting;
@@ -130,6 +122,14 @@ public class HeapMemoryManager {
     metricsHeapMemoryManager = new MetricsHeapMemoryManager();
   }
 
+  private ResizableBlockCache toResizableBlockCache(BlockCache blockCache) {
+    if (blockCache instanceof CombinedBlockCache) {
+      return (ResizableBlockCache) ((CombinedBlockCache) blockCache).getOnHeapCache();
+    } else {
+      return (ResizableBlockCache) blockCache;
+    }
+  }
+
   private boolean doInit(Configuration conf) {
     boolean tuningEnabled = true;
     globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
index b38c3e0..33a6ee0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
+import java.util.Optional;
 import java.util.OptionalDouble;
 import java.util.OptionalLong;
 import java.util.concurrent.ScheduledExecutorService;
@@ -32,9 +33,8 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.io.hfile.BlockCache;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.CacheStats;
-import org.apache.hadoop.hbase.mob.MobCacheConfig;
+import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
 import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -59,8 +59,11 @@ class MetricsRegionServerWrapperImpl
   private final HRegionServer regionServer;
   private final MetricsWALSource metricsWALSource;
 
-  private BlockCache blockCache;
-  private MobFileCache mobFileCache;
+  private Optional<BlockCache> blockCache;
+  private Optional<MobFileCache> mobFileCache;
+  private Optional<CacheStats> cacheStats;
+  private Optional<CacheStats> l1Stats = Optional.empty();
+  private Optional<CacheStats> l2Stats = Optional.empty();
 
   private volatile long numStores = 0;
   private volatile long numWALFiles = 0;
@@ -112,9 +115,6 @@ class MetricsRegionServerWrapperImpl
   private volatile long blockedRequestsCount = 0L;
   private volatile long averageRegionSize = 0L;
 
-  private CacheStats cacheStats;
-  private CacheStats l1Stats = null;
-  private CacheStats l2Stats = null;
   private ScheduledExecutorService executor;
   private Runnable runnable;
   private long period;
@@ -149,34 +149,26 @@ class MetricsRegionServerWrapperImpl
     }
   }
 
-  /**
-   * It's possible that due to threading the block cache could not be initialized
-   * yet (testing multiple region servers in one jvm).  So we need to try and initialize
-   * the blockCache and cacheStats reference multiple times until we succeed.
-   */
-  private synchronized  void initBlockCache() {
-    CacheConfig cacheConfig = this.regionServer.cacheConfig;
-    if (cacheConfig != null) {
-      l1Stats = cacheConfig.getOnHeapCacheStats();
-      l2Stats = cacheConfig.getL2CacheStats();
-      if (this.blockCache == null) {
-        this.blockCache = cacheConfig.getBlockCache();
+  private void initBlockCache() {
+    this.blockCache = this.regionServer.getBlockCache();
+    this.cacheStats = this.blockCache.map(BlockCache::getStats);
+    if (this.cacheStats.isPresent()) {
+      if (this.cacheStats.get() instanceof CombinedBlockCache.CombinedCacheStats) {
+        l1Stats = Optional
+            .of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get()).getLruCacheStats());
+        l2Stats = Optional.of(((CombinedBlockCache.CombinedCacheStats) this.cacheStats.get())
+            .getBucketCacheStats());
+      } else {
+        l1Stats = this.cacheStats;
       }
     }
-
-    if (this.blockCache != null && this.cacheStats == null) {
-      this.cacheStats = blockCache.getStats();
-    }
   }
 
   /**
    * Initializes the mob file cache.
    */
-  private synchronized void initMobFileCache() {
-    MobCacheConfig mobCacheConfig = this.regionServer.mobCacheConfig;
-    if (mobCacheConfig != null && this.mobFileCache == null) {
-      this.mobFileCache = mobCacheConfig.getMobFileCache();
-    }
+  private void initMobFileCache() {
+    this.mobFileCache = this.regionServer.getMobFileCache();
   }
 
   @Override
@@ -281,10 +273,7 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheCount() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getBlockCount();
+    return this.blockCache.map(BlockCache::getBlockCount).orElse(0L);
   }
 
   @Override
@@ -294,74 +283,47 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheSize() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getCurrentSize();
+    return this.blockCache.map(BlockCache::getCurrentSize).orElse(0L);
   }
 
   @Override
   public long getBlockCacheFreeSize() {
-    if (this.blockCache == null) {
-      return 0;
-    }
-    return this.blockCache.getFreeSize();
+    return this.blockCache.map(BlockCache::getFreeSize).orElse(0L);
   }
 
   @Override
   public long getBlockCacheHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getHitCount();
+    return this.cacheStats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryHitCount();
+    return this.cacheStats.map(CacheStats::getPrimaryHitCount).orElse(0L);
   }
 
   @Override
   public long getBlockCacheMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getMissCount();
+    return this.cacheStats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryMissCount();
+    return this.cacheStats.map(CacheStats::getPrimaryMissCount).orElse(0L);
   }
 
   @Override
   public long getBlockCacheEvictedCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getEvictedCount();
+    return this.cacheStats.map(CacheStats::getEvictedCount).orElse(0L);
   }
 
   @Override
   public long getBlockCachePrimaryEvictedCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getPrimaryEvictedCount();
+    return this.cacheStats.map(CacheStats::getPrimaryEvictedCount).orElse(0L);
   }
 
   @Override
   public double getBlockCacheHitPercent() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    double ratio = this.cacheStats.getHitRatio();
+    double ratio = this.cacheStats.map(CacheStats::getHitRatio).orElse(0.0);
     if (Double.isNaN(ratio)) {
       ratio = 0;
     }
@@ -370,12 +332,7 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public double getBlockCacheHitCachingPercent() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-
-    double ratio = this.cacheStats.getHitCachingRatio();
-
+    double ratio = this.cacheStats.map(CacheStats::getHitCachingRatio).orElse(0.0);
     if (Double.isNaN(ratio)) {
       ratio = 0;
     }
@@ -384,74 +341,47 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getBlockCacheFailedInsertions() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return this.cacheStats.getFailedInserts();
+    return this.cacheStats.map(CacheStats::getFailedInserts).orElse(0L);
   }
 
   @Override
   public long getL1CacheHitCount() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getHitCount();
+    return this.l1Stats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getL1CacheMissCount() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getMissCount();
+    return this.l1Stats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public double getL1CacheHitRatio() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getHitRatio();
+    return this.l1Stats.map(CacheStats::getHitRatio).orElse(0.0);
   }
 
   @Override
   public double getL1CacheMissRatio() {
-    if (this.l1Stats == null) {
-      return 0;
-    }
-    return this.l1Stats.getMissRatio();
+    return this.l1Stats.map(CacheStats::getMissRatio).orElse(0.0);
   }
 
   @Override
   public long getL2CacheHitCount() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getHitCount();
+    return this.l2Stats.map(CacheStats::getHitCount).orElse(0L);
   }
 
   @Override
   public long getL2CacheMissCount() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getMissCount();
+    return this.l2Stats.map(CacheStats::getMissCount).orElse(0L);
   }
 
   @Override
   public double getL2CacheHitRatio() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getHitRatio();
+    return this.l2Stats.map(CacheStats::getHitRatio).orElse(0.0);
   }
 
   @Override
   public double getL2CacheMissRatio() {
-    if (this.l2Stats == null) {
-      return 0;
-    }
-    return this.l2Stats.getMissRatio();
+    return this.l2Stats.map(CacheStats::getMissRatio).orElse(0.0);
   }
 
   @Override public void forceRecompute() {
@@ -741,9 +671,6 @@ class MetricsRegionServerWrapperImpl
     @Override
     synchronized public void run() {
       try {
-        initBlockCache();
-        initMobFileCache();
-
         HDFSBlocksDistribution hdfsBlocksDistribution =
             new HDFSBlocksDistribution();
         HDFSBlocksDistribution hdfsBlocksDistributionSecondaryRegions =
@@ -945,12 +872,14 @@ class MetricsRegionServerWrapperImpl
         mobFlushedCellsSize = tempMobFlushedCellsSize;
         mobScanCellsCount = tempMobScanCellsCount;
         mobScanCellsSize = tempMobScanCellsSize;
-        mobFileCacheAccessCount = mobFileCache.getAccessCount();
-        mobFileCacheMissCount = mobFileCache.getMissCount();
-        mobFileCacheHitRatio = Double.
-            isNaN(mobFileCache.getHitRatio())?0:mobFileCache.getHitRatio();
-        mobFileCacheEvictedCount = mobFileCache.getEvictedFileCount();
-        mobFileCacheCount = mobFileCache.getCacheSize();
+        mobFileCacheAccessCount = mobFileCache.map(MobFileCache::getAccessCount).orElse(0L);
+        mobFileCacheMissCount = mobFileCache.map(MobFileCache::getMissCount).orElse(0L);
+        mobFileCacheHitRatio = mobFileCache.map(MobFileCache::getHitRatio).orElse(0.0);
+        if (Double.isNaN(mobFileCacheHitRatio)) {
+          mobFileCacheHitRatio = 0.0;
+        }
+        mobFileCacheEvictedCount = mobFileCache.map(MobFileCache::getEvictedFileCount).orElse(0L);
+        mobFileCacheCount = mobFileCache.map(MobFileCache::getCacheSize).orElse(0);
         blockedRequestsCount = tempBlockedRequestsCount;
       } catch (Throwable e) {
         LOG.warn("Caught exception! Will suppress and retry.", e);
@@ -980,161 +909,101 @@ class MetricsRegionServerWrapperImpl
 
   @Override
   public long getDataMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDataMissCount();
+    return this.cacheStats.map(CacheStats::getDataMissCount).orElse(0L);
   }
 
   @Override
   public long getLeafIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getLeafIndexMissCount();
+    return this.cacheStats.map(CacheStats::getLeafIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getBloomChunkMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getBloomChunkMissCount();
+    return this.cacheStats.map(CacheStats::getBloomChunkMissCount).orElse(0L);
   }
 
   @Override
   public long getMetaMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getMetaMissCount();
+    return this.cacheStats.map(CacheStats::getMetaMissCount).orElse(0L);
   }
 
   @Override
   public long getRootIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getRootIndexMissCount();
+    return this.cacheStats.map(CacheStats::getRootIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getIntermediateIndexMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getIntermediateIndexMissCount();
+    return this.cacheStats.map(CacheStats::getIntermediateIndexMissCount).orElse(0L);
   }
 
   @Override
   public long getFileInfoMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getFileInfoMissCount();
+    return this.cacheStats.map(CacheStats::getFileInfoMissCount).orElse(0L);
   }
 
   @Override
   public long getGeneralBloomMetaMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getGeneralBloomMetaMissCount();
+    return this.cacheStats.map(CacheStats::getGeneralBloomMetaMissCount).orElse(0L);
   }
 
   @Override
   public long getDeleteFamilyBloomMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDeleteFamilyBloomMissCount();
+    return this.cacheStats.map(CacheStats::getDeleteFamilyBloomMissCount).orElse(0L);
   }
 
   @Override
   public long getTrailerMissCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getTrailerMissCount();
+    return this.cacheStats.map(CacheStats::getTrailerMissCount).orElse(0L);
   }
 
   @Override
   public long getDataHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDataHitCount();
+    return this.cacheStats.map(CacheStats::getDataHitCount).orElse(0L);
   }
 
   @Override
   public long getLeafIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getLeafIndexHitCount();
+    return this.cacheStats.map(CacheStats::getLeafIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getBloomChunkHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getBloomChunkHitCount();
+    return this.cacheStats.map(CacheStats::getBloomChunkHitCount).orElse(0L);
   }
 
   @Override
   public long getMetaHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getMetaHitCount();
+    return this.cacheStats.map(CacheStats::getMetaHitCount).orElse(0L);
   }
 
   @Override
   public long getRootIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getRootIndexHitCount();
+    return this.cacheStats.map(CacheStats::getRootIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getIntermediateIndexHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getIntermediateIndexHitCount();
+    return this.cacheStats.map(CacheStats::getIntermediateIndexHitCount).orElse(0L);
   }
 
   @Override
   public long getFileInfoHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getFileInfoHitCount();
+    return this.cacheStats.map(CacheStats::getFileInfoHitCount).orElse(0L);
   }
 
   @Override
   public long getGeneralBloomMetaHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getGeneralBloomMetaHitCount();
+    return this.cacheStats.map(CacheStats::getGeneralBloomMetaHitCount).orElse(0L);
   }
 
   @Override
   public long getDeleteFamilyBloomHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getDeleteFamilyBloomHitCount();
+    return this.cacheStats.map(CacheStats::getDeleteFamilyBloomHitCount).orElse(0L);
   }
 
   @Override
   public long getTrailerHitCount() {
-    if (this.cacheStats == null) {
-      return 0;
-    }
-    return cacheStats.getTrailerHitCount();
+    return this.cacheStats.map(CacheStats::getTrailerHitCount).orElse(0L);
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index df84dcf..31df37a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -89,6 +89,7 @@ import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
@@ -3683,7 +3684,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
         stats.addException(region.getRegionInfo().getRegionName(), e);
       }
     }
-    stats.withMaxCacheSize(regionServer.getCacheConfig().getBlockCache().getMaxSize());
+    stats.withMaxCacheSize(regionServer.getBlockCache().map(BlockCache::getMaxSize).orElse(0L));
     return builder.setStats(ProtobufUtil.toCacheEvictionStats(stats.build())).build();
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
index 37a3606..e0638ac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Server;
@@ -31,7 +32,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -266,4 +269,14 @@ public interface RegionServerServices extends Server, MutableOnlineRegions, Favo
    * @return Return table descriptors implementation.
    */
   TableDescriptors getTableDescriptors();
-}
+
+  /**
+   * @return The block cache instance.
+   */
+  Optional<BlockCache> getBlockCache();
+
+  /**
+   * @return The cache for mob files.
+   */
+  Optional<MobFileCache> getMobFileCache();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 31a7cad..0cd5a22 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -92,6 +92,7 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.io.hfile.ChecksumUtil;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
@@ -104,6 +105,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
 import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.ChunkCreator;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -2512,6 +2514,7 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     return new WALFactory(confForWAL, "hregion-" + RandomStringUtils.randomNumeric(8)).getWAL(hri);
   }
 
+
   /**
    * Create a region with it's own WAL. Be sure to call
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
@@ -2526,6 +2529,31 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
    */
   public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
+      final Configuration conf, final TableDescriptor htd, BlockCache blockCache)
+      throws IOException {
+    HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false);
+    region.setBlockCache(blockCache);
+    region.initialize();
+    return region;
+  }
+  /**
+   * Create a region with it's own WAL. Be sure to call
+   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
+   */
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
+      final Configuration conf, final TableDescriptor htd, MobFileCache mobFileCache)
+      throws IOException {
+    HRegion region = createRegionAndWAL(info, rootDir, conf, htd, false);
+    region.setMobFileCache(mobFileCache);
+    region.initialize();
+    return region;
+  }
+
+  /**
+   * Create a region with it's own WAL. Be sure to call
+   * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} to clean up all resources.
+   */
+  public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootDir,
       final Configuration conf, final TableDescriptor htd, boolean initialize)
       throws IOException {
     ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@@ -4037,17 +4065,21 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
       + " on server " + server);
   }
 
-  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd)
-      throws IOException {
-    TableDescriptor td
-        = TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName))
-            .setColumnFamily(cd)
-            .build();
-    HRegionInfo info =
-        new HRegionInfo(TableName.valueOf(tableName), null, null, false);
+  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
     return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td);
   }
 
+  public HRegion createTestRegion(String tableName, ColumnFamilyDescriptor cd,
+      BlockCache blockCache) throws IOException {
+    TableDescriptor td =
+        TableDescriptorBuilder.newBuilder(TableName.valueOf(tableName)).setColumnFamily(cd).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(TableName.valueOf(tableName)).build();
+    return createRegionAndWAL(info, getDataTestDir(), getConfiguration(), td, blockCache);
+  }
+
   public void setFileSystemURI(String fsURI) {
     FS_URI = fsURI;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
index ff0a88c..0e4f241 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
@@ -26,6 +26,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.hadoop.conf.Configuration;
@@ -36,7 +37,9 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.locking.EntityLock;
 import org.apache.hadoop.hbase.executor.ExecutorService;
 import org.apache.hadoop.hbase.fs.HFileSystem;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
+import org.apache.hadoop.hbase.mob.MobFileCache;
 import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
 import org.apache.hadoop.hbase.quotas.RegionSizeStore;
@@ -355,4 +358,14 @@ public class MockRegionServerServices implements RegionServerServices {
   public TableDescriptors getTableDescriptors() {
     return null;
   }
+
+  @Override
+  public Optional<BlockCache> getBlockCache() {
+    return Optional.empty();
+  }
+
+  @Override
+  public Optional<MobFileCache> getMobFileCache() {
+    return Optional.empty();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
index d22772a..31c01c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java
@@ -142,7 +142,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      final BlockCache cache = cacheConf.getBlockCache();
+      final BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 5 Rows are added
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -306,7 +306,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      final BlockCache cache = cacheConf.getBlockCache();
+      final BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 5 Rows are added
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index fc4c1f9..7f20195 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -196,7 +196,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -286,7 +286,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       insertData(table);
       // flush the data
@@ -345,7 +345,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -583,7 +583,7 @@ public class TestBlockEvictionFromClient {
       HStore store = region.getStores().iterator().next();
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -647,7 +647,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
@@ -803,7 +803,7 @@ public class TestBlockEvictionFromClient {
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
       // Use the last one
-      cache = cacheConf.getBlockCache();
+      cache = cacheConf.getBlockCache().get();
     }
     return cache;
   }
@@ -830,7 +830,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       insertData(table);
@@ -896,7 +896,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -1014,7 +1014,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // insert data. 2 Rows are added
       Put put = new Put(ROW);
@@ -1144,7 +1144,7 @@ public class TestBlockEvictionFromClient {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
       // insert data. 2 Rows are added
       insertData(table);
       // flush the data

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
index 65bc3f6..e5ffd73 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java
@@ -5234,14 +5234,13 @@ public class TestFromClientSide {
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setCacheDataOnWrite(true);
       cacheConf.setEvictOnClose(true);
-      BlockCache cache = cacheConf.getBlockCache();
+      BlockCache cache = cacheConf.getBlockCache().get();
 
       // establish baseline stats
       long startBlockCount = cache.getBlockCount();
       long startBlockHits = cache.getStats().getHitCount();
       long startBlockMiss = cache.getStats().getMissCount();
 
-
       // wait till baseline is stable, (minimal 500 ms)
       for (int i = 0; i < 5; i++) {
         Thread.sleep(100);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index 11d7bb4..350a316 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -28,15 +28,16 @@ import java.util.Map;
 import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
 import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -112,17 +113,17 @@ public class TestEncodedSeekers {
     if(includeTags) {
       testUtil.getConfiguration().setInt(HFile.FORMAT_VERSION_KEY, 3);
     }
-    CacheConfig.instantiateBlockCache(testUtil.getConfiguration());
+
     LruBlockCache cache =
-      (LruBlockCache)new CacheConfig(testUtil.getConfiguration()).getBlockCache();
-    cache.clearCache();
+        (LruBlockCache) BlockCacheFactory.createBlockCache(testUtil.getConfiguration());
     // Need to disable default row bloom filter for this test to pass.
-    HColumnDescriptor hcd = (new HColumnDescriptor(CF_NAME)).setMaxVersions(MAX_VERSIONS).
-        setDataBlockEncoding(encoding).
-        setBlocksize(BLOCK_SIZE).
-        setBloomFilterType(BloomType.NONE).
-        setCompressTags(compressTags);
-    HRegion region = testUtil.createTestRegion(TABLE_NAME, hcd);
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(CF_BYTES).setMaxVersions(MAX_VERSIONS).
+            setDataBlockEncoding(encoding).
+            setBlocksize(BLOCK_SIZE).
+            setBloomFilterType(BloomType.NONE).
+            setCompressTags(compressTags).build();
+    HRegion region = testUtil.createTestRegion(TABLE_NAME, cfd, cache);
 
     //write the data, but leave some in the memstore
     doPuts(region);
@@ -145,7 +146,6 @@ public class TestEncodedSeekers {
     assertTrue(encodingCounts.get(encodingInCache) > 0);
   }
 
-
   private void doPuts(HRegion region) throws IOException{
     LoadTestKVGenerator dataGenerator = new LoadTestKVGenerator(MIN_VALUE_SIZE, MAX_VALUE_SIZE);
      for (int i = 0; i < NUM_ROWS; ++i) {
@@ -175,7 +175,6 @@ public class TestEncodedSeekers {
     }
   }
 
-
   private void doGets(Region region) throws IOException{
     for (int i = 0; i < NUM_ROWS; ++i) {
       final byte[] rowKey = LoadTestKVGenerator.md5PrefixedKey(i).getBytes();
@@ -195,5 +194,4 @@ public class TestEncodedSeekers {
       }
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
index 19919e0..1313f31 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestBlockCacheReporting.java
@@ -17,12 +17,14 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Objects;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.DataCacheEntry;
 import org.apache.hadoop.hbase.io.hfile.TestCacheConfig.IndexCacheEntry;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -51,16 +52,9 @@ public class TestBlockCacheReporting {
 
   @Before
   public void setUp() throws Exception {
-    CacheConfig.clearGlobalInstances();
     this.conf = HBaseConfiguration.create();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    // Let go of current block cache.
-    CacheConfig.clearGlobalInstances();
-  }
-
   private void addDataAndHits(final BlockCache bc, final int count) {
     Cacheable dce = new DataCacheEntry();
     Cacheable ice = new IndexCacheEntry();
@@ -85,39 +79,36 @@ public class TestBlockCacheReporting {
   public void testBucketCache() throws IOException {
     this.conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, 100);
-    CacheConfig.instantiateBlockCache(this.conf);
-    CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
-    logPerBlock(cc.getBlockCache());
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    assertTrue(blockCache instanceof CombinedBlockCache);
+    logPerBlock(blockCache);
     final int count = 3;
-    addDataAndHits(cc.getBlockCache(), count);
+    addDataAndHits(blockCache, count);
     // The below has no asserts.  It is just exercising toString and toJSON code.
-    LOG.info(Objects.toString(cc.getBlockCache().getStats()));
-    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
+    LOG.info(Objects.toString(blockCache.getStats()));
+    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(blockCache);
     LOG.info(Objects.toString(cbsbf));
     logPerFile(cbsbf);
-    bucketCacheReport(cc.getBlockCache());
+    bucketCacheReport(blockCache);
     LOG.info(BlockCacheUtil.toJSON(cbsbf));
   }
 
   @Test
   public void testLruBlockCache() throws IOException {
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    logPerBlock(cc.getBlockCache());
-    addDataAndHits(cc.getBlockCache(), 3);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    assertTrue(blockCache instanceof LruBlockCache);
+    logPerBlock(blockCache);
+    addDataAndHits(blockCache, 3);
     // The below has no asserts.  It is just exercising toString and toJSON code.
-    BlockCache bc = cc.getBlockCache();
-    LOG.info("count=" + bc.getBlockCount() + ", currentSize=" + bc.getCurrentSize() +
-        ", freeSize=" + bc.getFreeSize() );
-    LOG.info(Objects.toString(cc.getBlockCache().getStats()));
-    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(cc.getBlockCache());
+    LOG.info("count=" + blockCache.getBlockCount() + ", currentSize=" + blockCache.getCurrentSize()
+        + ", freeSize=" + blockCache.getFreeSize());
+    LOG.info(Objects.toString(blockCache.getStats()));
+    BlockCacheUtil.CachedBlocksByFile cbsbf = logPerBlock(blockCache);
     LOG.info(Objects.toString(cbsbf));
     logPerFile(cbsbf);
-    bucketCacheReport(cc.getBlockCache());
+    bucketCacheReport(blockCache);
     LOG.info(BlockCacheUtil.toJSON(cbsbf));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
index 7b6bbb3..4c56fff 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheConfig.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Threads;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -165,33 +164,25 @@ public class TestCacheConfig {
 
   @Before
   public void setUp() throws Exception {
-    CacheConfig.clearGlobalInstances();
     this.conf = HBaseConfiguration.create();
   }
 
-  @After
-  public void tearDown() throws Exception {
-    // Let go of current block cache.
-    CacheConfig.clearGlobalInstances();
-  }
-
   /**
-   * @param cc
+   * @param bc The block cache instance.
+   * @param cc Cache config.
    * @param doubling If true, addition of element ups counter by 2, not 1, because element added
    * to onheap and offheap caches.
    * @param sizing True if we should run sizing test (doesn't always apply).
    */
-  void basicBlockCacheOps(final CacheConfig cc, final boolean doubling,
+  void basicBlockCacheOps(final BlockCache bc, final CacheConfig cc, final boolean doubling,
       final boolean sizing) {
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    BlockCache bc = cc.getBlockCache();
     BlockCacheKey bck = new BlockCacheKey("f", 0);
     Cacheable c = new DataCacheEntry();
     // Do asserts on block counting.
     long initialBlockCount = bc.getBlockCount();
     bc.cacheBlock(bck, c, cc.isInMemory());
-    assertEquals(doubling? 2: 1, bc.getBlockCount() - initialBlockCount);
+    assertEquals(doubling ? 2 : 1, bc.getBlockCount() - initialBlockCount);
     bc.evictBlock(bck);
     assertEquals(initialBlockCount, bc.getBlockCount());
     // Do size accounting.  Do it after the above 'warm-up' because it looks like some
@@ -209,7 +200,6 @@ public class TestCacheConfig {
   @Test
   public void testDisableCacheDataBlock() throws IOException {
     Configuration conf = HBaseConfiguration.create();
-    CacheConfig.instantiateBlockCache(conf);
     CacheConfig cacheConfig = new CacheConfig(conf);
     assertTrue(cacheConfig.shouldCacheBlockOnRead(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheCompressed(BlockCategory.DATA));
@@ -260,7 +250,7 @@ public class TestCacheConfig {
     HColumnDescriptor family = new HColumnDescriptor("testDisableCacheDataBlock");
     family.setBlockCacheEnabled(false);
 
-    cacheConfig = new CacheConfig(conf, family);
+    cacheConfig = new CacheConfig(conf, family, null);
     assertFalse(cacheConfig.shouldCacheBlockOnRead(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheCompressed(BlockCategory.DATA));
     assertFalse(cacheConfig.shouldCacheDataCompressed());
@@ -275,12 +265,11 @@ public class TestCacheConfig {
 
   @Test
   public void testCacheConfigDefaultLRUBlockCache() {
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    assertTrue(cc.isBlockCacheEnabled());
     assertTrue(CacheConfig.DEFAULT_IN_MEMORY == cc.isInMemory());
-    basicBlockCacheOps(cc, false, true);
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, true);
+    assertTrue(blockCache instanceof LruBlockCache);
   }
 
   /**
@@ -309,18 +298,18 @@ public class TestCacheConfig {
   private void doBucketCacheConfigTest() {
     final int bcSize = 100;
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    basicBlockCacheOps(cc, false, false);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, false);
+    assertTrue(blockCache instanceof CombinedBlockCache);
     // TODO: Assert sizes allocated are right and proportions.
-    CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
-    BlockCache [] bcs = cbc.getBlockCaches();
+    CombinedBlockCache cbc = (CombinedBlockCache) blockCache;
+    BlockCache[] bcs = cbc.getBlockCaches();
     assertTrue(bcs[0] instanceof LruBlockCache);
-    LruBlockCache lbc = (LruBlockCache)bcs[0];
+    LruBlockCache lbc = (LruBlockCache) bcs[0];
     assertEquals(MemorySizeUtil.getOnHeapCacheSize(this.conf), lbc.getMaxSize());
     assertTrue(bcs[1] instanceof BucketCache);
-    BucketCache bc = (BucketCache)bcs[1];
+    BucketCache bc = (BucketCache) bcs[1];
     // getMaxSize comes back in bytes but we specified size in MB
     assertEquals(bcSize, bc.getMaxSize() / (1024 * 1024));
   }
@@ -341,12 +330,12 @@ public class TestCacheConfig {
     long bcExpectedSize = 100 * 1024 * 1024; // MB.
     assertTrue(lruExpectedSize < bcExpectedSize);
     this.conf.setInt(HConstants.BUCKET_CACHE_SIZE_KEY, bcSize);
-    CacheConfig.instantiateBlockCache(this.conf);
     CacheConfig cc = new CacheConfig(this.conf);
-    basicBlockCacheOps(cc, false, false);
-    assertTrue(cc.getBlockCache() instanceof CombinedBlockCache);
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(this.conf);
+    basicBlockCacheOps(blockCache, cc, false, false);
+    assertTrue(blockCache instanceof CombinedBlockCache);
     // TODO: Assert sizes allocated are right and proportions.
-    CombinedBlockCache cbc = (CombinedBlockCache)cc.getBlockCache();
+    CombinedBlockCache cbc = (CombinedBlockCache) blockCache;
     LruBlockCache lbc = cbc.onHeapCache;
     assertEquals(lruExpectedSize, lbc.getMaxSize());
     BlockCache bc = cbc.l2Cache;
@@ -382,10 +371,10 @@ public class TestCacheConfig {
   public void testL2CacheWithInvalidBucketSize() {
     Configuration c = new Configuration(this.conf);
     c.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap");
-    c.set(CacheConfig.BUCKET_CACHE_BUCKETS_KEY, "256,512,1024,2048,4000,4096");
+    c.set(BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY, "256,512,1024,2048,4000,4096");
     c.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 1024);
     try {
-      CacheConfig.getBucketCache(c);
+      BlockCacheFactory.createBlockCache(c);
       fail("Should throw IllegalArgumentException when passing illegal value for bucket size");
     } catch (IllegalArgumentException e) {
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index 9c2f6df..4163d55 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -31,6 +31,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -39,10 +40,11 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -51,7 +53,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -160,8 +161,7 @@ public class TestCacheOnWrite {
     Configuration conf = TEST_UTIL.getConfiguration();
     List<BlockCache> blockcaches = new ArrayList<>();
     // default
-    CacheConfig.instantiateBlockCache(conf);
-    blockcaches.add(new CacheConfig(conf).getBlockCache());
+    blockcaches.add(BlockCacheFactory.createBlockCache(conf));
 
     //set LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME to 2.0f due to HBASE-16287
     TEST_UTIL.getConfiguration().setFloat(LruBlockCache.LRU_HARD_CAPACITY_LIMIT_FACTOR_CONFIG_NAME, 2.0f);
@@ -224,16 +224,16 @@ public class TestCacheOnWrite {
     conf = TEST_UTIL.getConfiguration();
     this.conf.set("dfs.datanode.data.dir.perm", "700");
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, INDEX_BLOCK_SIZE);
-    conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
-        BLOOM_BLOCK_SIZE);
+    conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, BLOOM_BLOCK_SIZE);
     conf.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, cacheCompressedData);
     cowType.modifyConf(conf);
+    conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, cowType.shouldBeCached(BlockType.DATA));
+    conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.LEAF_INDEX));
+    conf.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY,
+        cowType.shouldBeCached(BlockType.BLOOM_CHUNK));
+    cacheConf = new CacheConfig(conf, blockCache);
     fs = HFileSystem.get(conf);
-    cacheConf =
-        new CacheConfig(blockCache, true, true, cowType.shouldBeCached(BlockType.DATA),
-        cowType.shouldBeCached(BlockType.LEAF_INDEX),
-        cowType.shouldBeCached(BlockType.BLOOM_CHUNK), false, cacheCompressedData,
-            false, false);
   }
 
   @After
@@ -414,13 +414,11 @@ public class TestCacheOnWrite {
     final String cf = "myCF";
     final byte[] cfBytes = Bytes.toBytes(cf);
     final int maxVersions = 3;
-    HRegion region = TEST_UTIL.createTestRegion(table,
-        new HColumnDescriptor(cf)
-            .setCompressionType(compress)
-            .setBloomFilterType(BLOOM_TYPE)
-            .setMaxVersions(maxVersions)
-            .setDataBlockEncoding(NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding())
-    );
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(cfBytes).setCompressionType(compress)
+            .setBloomFilterType(BLOOM_TYPE).setMaxVersions(maxVersions)
+            .setDataBlockEncoding(NoOpDataBlockEncoder.INSTANCE.getDataBlockEncoding()).build();
+    HRegion region = TEST_UTIL.createTestRegion(table, cfd, blockCache);
     int rowIdx = 0;
     long ts = EnvironmentEdgeManager.currentTime();
     for (int iFile = 0; iFile < 5; ++iFile) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
index 5612c1b..fd39f48 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestForceCacheImportantBlocks.java
@@ -25,14 +25,14 @@ import java.util.Arrays;
 import java.util.Collection;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -103,24 +103,20 @@ public class TestForceCacheImportantBlocks {
 
   @Before
   public void setup() {
-    // Make sure we make a new one each time.
-    CacheConfig.clearGlobalInstances();
     HFile.DATABLOCK_READ_COUNT.reset();
-    CacheConfig.instantiateBlockCache(TEST_UTIL.getConfiguration());
   }
 
   @Test
   public void testCacheBlocks() throws IOException {
     // Set index block size to be the same as normal block size.
     TEST_UTIL.getConfiguration().setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, BLOCK_SIZE);
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes(CF)).setMaxVersions(MAX_VERSIONS).
-      setCompressionType(COMPRESSION_ALGORITHM).
-      setBloomFilterType(BLOOM_TYPE);
-    hcd.setBlocksize(BLOCK_SIZE);
-    hcd.setBlockCacheEnabled(cfCacheEnabled);
-    HRegion region = TEST_UTIL.createTestRegion(TABLE, hcd);
-    BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache();
-    CacheStats stats = cache.getStats();
+    BlockCache blockCache = BlockCacheFactory.createBlockCache(TEST_UTIL.getConfiguration());
+    ColumnFamilyDescriptor cfd =
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(CF)).setMaxVersions(MAX_VERSIONS)
+            .setCompressionType(COMPRESSION_ALGORITHM).setBloomFilterType(BLOOM_TYPE)
+            .setBlocksize(BLOCK_SIZE).setBlockCacheEnabled(cfCacheEnabled).build();
+    HRegion region = TEST_UTIL.createTestRegion(TABLE, cfd, blockCache);
+    CacheStats stats = blockCache.getStats();
     writeTestData(region);
     assertEquals(0, stats.getHitCount());
     assertEquals(0, HFile.DATABLOCK_READ_COUNT.sum());

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 7053fce..2a613de 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -86,13 +86,14 @@ public class TestHFile  {
     TEST_UTIL.getDataTestDir("TestHFile").toString();
   private final int minBlockSize = 512;
   private static String localFormatter = "%010d";
-  private static CacheConfig cacheConf = null;
+  private static CacheConfig cacheConf;
   private static Configuration conf ;
   private static FileSystem fs;
 
   @BeforeClass
   public static void setUp() throws Exception {
     conf = TEST_UTIL.getConfiguration();
+    cacheConf = new CacheConfig(conf);
     fs = TEST_UTIL.getTestFileSystem();
   }
 
@@ -162,7 +163,6 @@ public class TestHFile  {
    */
   @Test
   public void testEmptyHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     HFileContext context = new HFileContextBuilder().withIncludesTags(false).build();
     Writer w =
@@ -179,7 +179,6 @@ public class TestHFile  {
    */
   @Test
   public void testCorrupt0LengthHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     FSDataOutputStream fsos = fs.create(f);
     fsos.close();
@@ -213,7 +212,6 @@ public class TestHFile  {
    */
   @Test
   public void testCorruptTruncatedHFile() throws IOException {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path f = new Path(ROOT_DIR, testName.getMethodName());
     HFileContext  context = new HFileContextBuilder().build();
     Writer w = HFile.getWriterFactory(conf, cacheConf).withPath(this.fs, f)
@@ -315,7 +313,6 @@ public class TestHFile  {
     if (useTags) {
       conf.setInt("hfile.format.version", 3);
     }
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path  ncHFile = new Path(ROOT_DIR, "basic.hfile." + codec.toString() + useTags);
     FSDataOutputStream fout = createFSOutput(ncHFile);
     HFileContext meta = new HFileContextBuilder()
@@ -411,7 +408,6 @@ public class TestHFile  {
   }
 
   private void metablocks(final String compress) throws Exception {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     Path mFile = new Path(ROOT_DIR, "meta.hfile");
     FSDataOutputStream fout = createFSOutput(mFile);
     HFileContext meta = new HFileContextBuilder()
@@ -445,7 +441,6 @@ public class TestHFile  {
 
   @Test
   public void testNullMetaBlocks() throws Exception {
-    if (cacheConf == null) cacheConf = new CacheConfig(conf);
     for (Compression.Algorithm compressAlgo :
         HBaseCommonTestingUtility.COMPRESSION_ALGORITHMS) {
       Path mFile = new Path(ROOT_DIR, "nometa_" + compressAlgo + ".hfile");

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index a588341..48080b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -32,6 +32,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Random;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -246,7 +247,7 @@ public class TestHFileBlock {
   @Test
   public void testNoCompression() throws IOException {
     CacheConfig cacheConf = Mockito.mock(CacheConfig.class);
-    Mockito.when(cacheConf.isBlockCacheEnabled()).thenReturn(false);
+    Mockito.when(cacheConf.getBlockCache()).thenReturn(Optional.empty());
 
     HFileBlock block =
       createTestV2Block(NONE, includesMemstoreTS, false).getBlockForCaching(cacheConf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
index efe76aa..78f8584 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java
@@ -530,9 +530,8 @@ public class TestHFileBlockIndex {
     conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, maxChunkSize);
     // should open hfile.block.index.cacheonwrite
     conf.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, true);
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    BlockCache blockCache = cacheConf.getBlockCache();
+    CacheConfig cacheConf = new CacheConfig(conf, BlockCacheFactory.createBlockCache(conf));
+    BlockCache blockCache = cacheConf.getBlockCache().get();
     // Evict all blocks that were cached-on-write by the previous invocation.
     blockCache.evictBlocksByHfileName(hfilePath.getName());
     // Write the HFile
@@ -589,9 +588,8 @@ public class TestHFileBlockIndex {
   public void testHFileWriterAndReader() throws IOException {
     Path hfilePath = new Path(TEST_UTIL.getDataTestDir(),
         "hfile_for_block_index");
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    BlockCache blockCache = cacheConf.getBlockCache();
+    CacheConfig cacheConf = new CacheConfig(conf, BlockCacheFactory.createBlockCache(conf));
+    BlockCache blockCache = cacheConf.getBlockCache().get();
 
     for (int testI = 0; testI < INDEX_CHUNK_SIZES.length; ++testI) {
       int indexBlockSize = INDEX_CHUNK_SIZES[testI];

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
index 4542a3c..5935f91 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestLazyDataBlockDecompression.java
@@ -82,13 +82,11 @@ public class TestLazyDataBlockDecompression {
 
   @Before
   public void setUp() throws IOException {
-    CacheConfig.clearGlobalInstances();
     fs = FileSystem.get(TEST_UTIL.getConfiguration());
   }
 
   @After
   public void tearDown() {
-    CacheConfig.clearGlobalInstances();
     fs = null;
   }
 
@@ -159,12 +157,11 @@ public class TestLazyDataBlockDecompression {
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressDisabled.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, false);
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE =
-      new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressDisabled);
-    CacheConfig cc = new CacheConfig(lazyCompressDisabled);
+    CacheConfig cc = new CacheConfig(lazyCompressDisabled,
+        new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressDisabled));
     assertFalse(cc.shouldCacheDataCompressed());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    LruBlockCache disabledBlockCache = (LruBlockCache) cc.getBlockCache();
+    assertTrue(cc.getBlockCache().get() instanceof LruBlockCache);
+    LruBlockCache disabledBlockCache = (LruBlockCache) cc.getBlockCache().get();
     LOG.info("disabledBlockCache=" + disabledBlockCache);
     assertEquals("test inconsistency detected.", maxSize, disabledBlockCache.getMaxSize());
     assertTrue("eviction thread spawned unintentionally.",
@@ -194,12 +191,11 @@ public class TestLazyDataBlockDecompression {
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_BLOOM_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_INDEX_BLOCKS_ON_WRITE_KEY, cacheOnWrite);
     lazyCompressEnabled.setBoolean(CacheConfig.CACHE_DATA_BLOCKS_COMPRESSED_KEY, true);
-    CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE =
-      new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressEnabled);
-    cc = new CacheConfig(lazyCompressEnabled);
+    cc = new CacheConfig(lazyCompressEnabled,
+        new LruBlockCache(maxSize, HConstants.DEFAULT_BLOCKSIZE, false, lazyCompressEnabled));
     assertTrue("test improperly configured.", cc.shouldCacheDataCompressed());
-    assertTrue(cc.getBlockCache() instanceof LruBlockCache);
-    LruBlockCache enabledBlockCache = (LruBlockCache) cc.getBlockCache();
+    assertTrue(cc.getBlockCache().get() instanceof LruBlockCache);
+    LruBlockCache enabledBlockCache = (LruBlockCache) cc.getBlockCache().get();
     LOG.info("enabledBlockCache=" + enabledBlockCache);
     assertEquals("test inconsistency detected", maxSize, enabledBlockCache.getMaxSize());
     assertTrue("eviction thread spawned unintentionally.",

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
index 811df14..9986bba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java
@@ -17,10 +17,12 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.Random;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -28,8 +30,9 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.testclassification.IOTests;
@@ -57,24 +60,25 @@ public class TestPrefetch {
   private Configuration conf;
   private CacheConfig cacheConf;
   private FileSystem fs;
+  private BlockCache blockCache;
 
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true);
     fs = HFileSystem.get(conf);
-    CacheConfig.blockCacheDisabled = false;
-    CacheConfig.instantiateBlockCache(conf);
-    cacheConf = new CacheConfig(conf);
+    blockCache = BlockCacheFactory.createBlockCache(conf);
+    cacheConf = new CacheConfig(conf, blockCache);
   }
 
   @Test
   public void testPrefetchSetInHCDWorks() {
-    HColumnDescriptor hcd = new HColumnDescriptor(Bytes.toBytes("f"));
-    hcd.setPrefetchBlocksOnOpen(true);
+    ColumnFamilyDescriptor columnFamilyDescriptor =
+        ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes("f")).setPrefetchBlocksOnOpen(true)
+            .build();
     Configuration c = HBaseConfiguration.create();
     assertFalse(c.getBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, false));
-    CacheConfig cc = new CacheConfig(c, hcd);
+    CacheConfig cc = new CacheConfig(c, columnFamilyDescriptor, blockCache);
     assertTrue(cc.shouldPrefetchOnOpen());
   }
 
@@ -119,7 +123,7 @@ public class TestPrefetch {
     }
 
     // Check that all of the data blocks were preloaded
-    BlockCache blockCache = cacheConf.getBlockCache();
+    BlockCache blockCache = cacheConf.getBlockCache().get();
     long offset = 0;
     while (offset < reader.getTrailer().getLoadOnOpenDataOffset()) {
       HFileBlock block = reader.readBlock(offset, -1, false, true, false, true, null, null);

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
index 18e8e70..ad6a0ab 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerFromBucketCache.java
@@ -25,29 +25,28 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.ByteBufferKeyValue;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.regionserver.ChunkCreator;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
-import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
-import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -88,7 +87,6 @@ public class TestScannerFromBucketCache {
       conf.setFloat("hbase.regionserver.global.memstore.size", 0.1f);
     }
     tableName = TableName.valueOf(name.getMethodName());
-    CacheConfig.instantiateBlockCache(conf);
   }
 
   @After
@@ -96,7 +94,6 @@ public class TestScannerFromBucketCache {
     EnvironmentEdgeManagerTestHelper.reset();
     LOG.info("Cleaning test directory: " + test_util.getDataTestDir());
     test_util.cleanupTestDir();
-    CacheConfig.clearGlobalInstances();
   }
 
   String getName() {
@@ -210,9 +207,7 @@ public class TestScannerFromBucketCache {
       Thread.sleep(500);
       // do the scan again and verify. This time it should be from the bucket cache in offheap mode
       // but one of the cell will be copied due to the asSubByteBuff call
-      Scan scan = new Scan(row1);
-      scan.addFamily(fam1);
-      scan.setMaxVersions(10);
+      Scan scan = new Scan().withStartRow(row1).addFamily(fam1).readVersions(10);
       actual = new ArrayList<>();
       InternalScanner scanner = region.getScanner(scan);
 
@@ -290,9 +285,7 @@ public class TestScannerFromBucketCache {
   }
 
   private List<Cell> performScan(byte[] row1, byte[] fam1) throws IOException {
-    Scan scan = new Scan(row1);
-    scan.addFamily(fam1);
-    scan.setMaxVersions(MAX_VERSIONS);
+    Scan scan = new Scan().withStartRow(row1).addFamily(fam1).readVersions(MAX_VERSIONS);
     List<Cell> actual = new ArrayList<>();
     InternalScanner scanner = region.getScanner(scan);
 
@@ -307,32 +300,19 @@ public class TestScannerFromBucketCache {
   }
 
   private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
-      String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
+      String callingMethod, Configuration conf, HBaseTestingUtility testUtil, boolean isReadOnly,
       byte[]... families) throws IOException {
-    ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
-    Path logDir = test_util.getDataTestDirOnTestFS(callingMethod + ".log");
-    HRegionInfo hri = new HRegionInfo(tableName, startKey, stopKey);
-    final WAL wal = HBaseTestingUtility.createWal(conf, logDir, hri);
-    return initHRegion(tableName, startKey, stopKey, callingMethod, conf, test_util, isReadOnly,
-      Durability.SYNC_WAL, wal, families);
-  }
-
-  /**
-   * @param tableName
-   * @param startKey
-   * @param stopKey
-   * @param callingMethod
-   * @param conf
-   * @param isReadOnly
-   * @param families
-   * @throws IOException
-   * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)}
-   *         when done.
-   */
-  private static HRegion initHRegion(TableName tableName, byte[] startKey, byte[] stopKey,
-      String callingMethod, Configuration conf, HBaseTestingUtility test_util, boolean isReadOnly,
-      Durability durability, WAL wal, byte[]... families) throws IOException {
-    return test_util.createLocalHRegion(tableName, startKey, stopKey, isReadOnly, durability, wal,
-      families);
+    RegionInfo regionInfo =
+        RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).setEndKey(stopKey).build();
+    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(tableName);
+    builder.setReadOnly(isReadOnly).setDurability(Durability.SYNC_WAL);
+    for (byte[] family : families) {
+      builder.setColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(Integer.MAX_VALUE)
+              .build());
+    }
+    return HBaseTestingUtility
+        .createRegionAndWAL(regionInfo, testUtil.getDataTestDir(callingMethod), conf,
+            builder.build(), BlockCacheFactory.createBlockCache(conf));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1971d02e/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
index d27b041..ed440e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingKeyRange.java
@@ -123,10 +123,7 @@ public class TestScannerSelectionUsingKeyRange {
     }
 
     Scan scan = new Scan(Bytes.toBytes("aaa"), Bytes.toBytes("aaz"));
-    CacheConfig.blockCacheDisabled = false;
-    CacheConfig.instantiateBlockCache(conf);
-    CacheConfig cacheConf = new CacheConfig(conf);
-    LruBlockCache cache = (LruBlockCache) cacheConf.getBlockCache();
+    LruBlockCache cache = (LruBlockCache) BlockCacheFactory.createBlockCache(conf);
     cache.clearCache();
     InternalScanner scanner = region.getScanner(scan);
     List<Cell> results = new ArrayList<>();


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

Posted by zg...@apache.org.
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/master
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);