Skip to content

Commit 317d3aa

Browse files
authored
HBASE-28724 BucketCache.notifyFileCachingCompleted may throw IllegalMonitorStateException (#6074) (#6081)
Signed-off-by: Peter Somogyi <psomogyi@apache.org>
1 parent 5a19920 commit 317d3aa

File tree

3 files changed

+71
-8
lines changed

3 files changed

+71
-8
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2092,6 +2092,7 @@ public void notifyFileCachingCompleted(Path fileName, int totalBlockCount, int d
20922092
for (ReentrantReadWriteLock lock : locks) {
20932093
lock.readLock().unlock();
20942094
}
2095+
locks.clear();
20952096
LOG.debug("There are still blocks pending caching for file {}. Will sleep 100ms "
20962097
+ "and try the verification again.", fileName.getName());
20972098
Thread.sleep(100);

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/CacheTestUtils.java

Lines changed: 17 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@
3232
import java.util.concurrent.ThreadLocalRandom;
3333
import java.util.concurrent.atomic.AtomicInteger;
3434
import org.apache.hadoop.conf.Configuration;
35+
import org.apache.hadoop.fs.Path;
3536
import org.apache.hadoop.hbase.HConstants;
3637
import org.apache.hadoop.hbase.MultithreadedTestUtil;
3738
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
@@ -275,6 +276,10 @@ public BlockType getBlockType() {
275276
}
276277

277278
public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
279+
return generateBlocksForPath(blockSize, numBlocks, null);
280+
}
281+
282+
public static HFileBlockPair[] generateBlocksForPath(int blockSize, int numBlocks, Path path) {
278283
HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
279284
Random rand = ThreadLocalRandom.current();
280285
HashSet<String> usedStrings = new HashSet<>();
@@ -299,16 +304,20 @@ public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks)
299304
prevBlockOffset, ByteBuff.wrap(cachedBuffer), HFileBlock.DONT_FILL_HEADER, blockSize,
300305
onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta,
301306
ByteBuffAllocator.HEAP);
302-
303-
String strKey;
304-
/* No conflicting keys */
305-
strKey = Long.toString(rand.nextLong());
306-
while (!usedStrings.add(strKey)) {
307-
strKey = Long.toString(rand.nextLong());
307+
String key = null;
308+
long offset = 0;
309+
if (path != null) {
310+
key = path.getName();
311+
offset = i * blockSize;
312+
} else {
313+
/* No conflicting keys */
314+
key = Long.toString(rand.nextLong());
315+
while (!usedStrings.add(key)) {
316+
key = Long.toString(rand.nextLong());
317+
}
308318
}
309-
310319
returnedBlocks[i] = new HFileBlockPair();
311-
returnedBlocks[i].blockName = new BlockCacheKey(strKey, 0);
320+
returnedBlocks[i].blockName = new BlockCacheKey(key, offset);
312321
returnedBlocks[i].block = generated;
313322
}
314323
return returnedBlocks;

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/bucket/TestBucketCache.java

Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -890,4 +890,57 @@ public void testBlockAdditionWaitWhenCache() throws Exception {
890890
HBASE_TESTING_UTILITY.cleanupTestDir();
891891
}
892892
}
893+
894+
@Test
895+
public void testNotifyFileCachingCompletedSuccess() throws Exception {
896+
BucketCache bucketCache = null;
897+
try {
898+
Path filePath =
899+
new Path(HBASE_TESTING_UTILITY.getDataTestDir(), "testNotifyFileCachingCompletedSuccess");
900+
bucketCache = testNotifyFileCachingCompleted(filePath, 10);
901+
assertTrue(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
902+
} finally {
903+
if (bucketCache != null) {
904+
bucketCache.shutdown();
905+
}
906+
HBASE_TESTING_UTILITY.cleanupTestDir();
907+
}
908+
}
909+
910+
@Test
911+
public void testNotifyFileCachingCompletedNotAllCached() throws Exception {
912+
BucketCache bucketCache = null;
913+
try {
914+
Path filePath = new Path(HBASE_TESTING_UTILITY.getDataTestDir(),
915+
"testNotifyFileCachingCompletedNotAllCached");
916+
// Deliberately passing more blocks than we have created to test that
917+
// notifyFileCachingCompleted will not consider the file fully cached
918+
bucketCache = testNotifyFileCachingCompleted(filePath, 12);
919+
assertFalse(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
920+
} finally {
921+
if (bucketCache != null) {
922+
bucketCache.shutdown();
923+
}
924+
HBASE_TESTING_UTILITY.cleanupTestDir();
925+
}
926+
}
927+
928+
private BucketCache testNotifyFileCachingCompleted(Path filePath, int totalBlocks)
929+
throws Exception {
930+
final Path dataTestDir = createAndGetTestDir();
931+
String ioEngineName = "file:" + dataTestDir + "/bucketNoRecycler.cache";
932+
BucketCache bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
933+
constructedBlockSizes, 1, 1, null);
934+
long usedByteSize = bucketCache.getAllocator().getUsedSize();
935+
assertEquals(0, usedByteSize);
936+
HFileBlockPair[] hfileBlockPairs =
937+
CacheTestUtils.generateBlocksForPath(constructedBlockSize, 10, filePath);
938+
// Add blocks
939+
for (HFileBlockPair hfileBlockPair : hfileBlockPairs) {
940+
bucketCache.cacheBlock(hfileBlockPair.getBlockName(), hfileBlockPair.getBlock(), false, true);
941+
}
942+
bucketCache.notifyFileCachingCompleted(filePath, totalBlocks, totalBlocks,
943+
totalBlocks * constructedBlockSize);
944+
return bucketCache;
945+
}
893946
}

0 commit comments

Comments
 (0)