You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2023/05/12 09:43:39 UTC
[hbase] branch master updated: HBASE-27852: Interrupt BucketCachePersister thread when BucketCache is shutdown (#5230)
This is an automated email from the ASF dual-hosted git repository.
wchevreuil pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new e343584b501 HBASE-27852: Interrupt BucketCachePersister thread when BucketCache is shutdown (#5230)
e343584b501 is described below
commit e343584b5015958015e97bc0a25650bf9399b3b2
Author: Kota-SH <sh...@gmail.com>
AuthorDate: Fri May 12 04:43:27 2023 -0500
HBASE-27852: Interrupt BucketCachePersister thread when BucketCache is shutdown (#5230)
Signed-off-by: Wellington Chevreuil <wc...@apache.org>
---
.../hadoop/hbase/io/hfile/bucket/BucketCache.java | 6 ++--
.../io/hfile/bucket/BucketCachePersister.java | 7 ++--
.../io/hfile/bucket/TestVerifyBucketCacheFile.java | 40 ++++++++++++++++++++++
3 files changed, 49 insertions(+), 4 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
index 8c9a24b79b7..14c4c44ee16 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java
@@ -178,6 +178,8 @@ public class BucketCache implements BlockCache, HeapSize {
private final BucketCacheStats cacheStats = new BucketCacheStats();
+ /** BucketCache persister thread */
+ private BucketCachePersister cachePersister;
private final String persistencePath;
static AtomicBoolean isCacheInconsistent = new AtomicBoolean(false);
private final long cacheCapacity;
@@ -377,8 +379,7 @@ public class BucketCache implements BlockCache, HeapSize {
}
void startBucketCachePersisterThread() {
- BucketCachePersister cachePersister =
- new BucketCachePersister(this, bucketcachePersistInterval);
+ cachePersister = new BucketCachePersister(this, bucketcachePersistInterval);
cachePersister.setDaemon(true);
cachePersister.start();
}
@@ -1416,6 +1417,7 @@ public class BucketCache implements BlockCache, HeapSize {
LOG.info("Shutdown bucket cache: IO persistent=" + ioEngine.isPersistent() + "; path to write="
+ persistencePath);
if (ioEngine.isPersistent() && persistencePath != null) {
+ cachePersister.interrupt();
try {
join();
persistToFile();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCachePersister.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCachePersister.java
index 099a19db0a1..dbea4f3f325 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCachePersister.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCachePersister.java
@@ -44,8 +44,11 @@ public class BucketCachePersister extends Thread {
cache.persistToFile();
cache.setCacheInconsistent(false);
}
- } catch (IOException | InterruptedException e) {
- LOG.warn("Exception in BucketCachePersister" + e.getMessage());
+ } catch (IOException e) {
+ LOG.warn("IOException in BucketCachePersister {} ", e.getMessage());
+ } catch (InterruptedException iex) {
+ LOG.warn("InterruptedException in BucketCachePersister {} ", iex.getMessage());
+ break;
}
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestVerifyBucketCacheFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestVerifyBucketCacheFile.java
index c720de22f43..3b2b9961b2b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestVerifyBucketCacheFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestVerifyBucketCacheFile.java
@@ -29,10 +29,12 @@ import java.nio.file.Files;
import java.nio.file.attribute.FileTime;
import java.time.Instant;
import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
import org.apache.hadoop.hbase.io.hfile.Cacheable;
import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -143,6 +145,44 @@ public class TestVerifyBucketCacheFile {
TEST_UTIL.cleanupTestDir();
}
+ @Test
+ public void testRetrieveFromFileAfterDelete() throws Exception {
+
+ HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
+ Path testDir = TEST_UTIL.getDataTestDir();
+ TEST_UTIL.getTestFileSystem().mkdirs(testDir);
+ Configuration conf = TEST_UTIL.getConfiguration();
+ conf.setLong(CacheConfig.BUCKETCACHE_PERSIST_INTERVAL_KEY, 300);
+
+ BucketCache bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize,
+ constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen,
+ testDir + "/bucket.persistence", 60 * 1000, conf);
+
+ long usedSize = bucketCache.getAllocator().getUsedSize();
+ assertEquals(0, usedSize);
+ CacheTestUtils.HFileBlockPair[] blocks =
+ CacheTestUtils.generateHFileBlocks(constructedBlockSize, 1);
+ // Add blocks
+ for (CacheTestUtils.HFileBlockPair block : blocks) {
+ cacheAndWaitUntilFlushedToBucket(bucketCache, block.getBlockName(), block.getBlock());
+ }
+ usedSize = bucketCache.getAllocator().getUsedSize();
+ assertNotEquals(0, usedSize);
+ // Shutdown BucketCache
+ bucketCache.shutdown();
+ // Delete the persistence file
+ final java.nio.file.Path mapFile =
+ FileSystems.getDefault().getPath(testDir.toString(), "bucket.persistence");
+ assertTrue(Files.deleteIfExists(mapFile));
+ Thread.sleep(350);
+ // Create BucketCache
+ bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize,
+ constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen,
+ testDir + "/bucket.persistence", 60 * 1000, conf);
+ assertEquals(0, bucketCache.getAllocator().getUsedSize());
+ assertEquals(0, bucketCache.backingMap.size());
+ }
+
/**
* Test whether BucketCache is started normally after modifying the cache file. Start BucketCache
* and add some blocks, then shutdown BucketCache and persist cache to file. Restart BucketCache