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#<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);