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

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

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)