Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2092,6 +2092,7 @@ public void notifyFileCachingCompleted(Path fileName, int totalBlockCount, int d
for (ReentrantReadWriteLock lock : locks) {
lock.readLock().unlock();
}
locks.clear();
LOG.debug("There are still blocks pending caching for file {}. Will sleep 100ms "
+ "and try the verification again.", fileName.getName());
Thread.sleep(100);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MultithreadedTestUtil;
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
Expand Down Expand Up @@ -275,6 +276,10 @@ public BlockType getBlockType() {
}

public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks) {
return generateBlocksForPath(blockSize, numBlocks, null);
}

public static HFileBlockPair[] generateBlocksForPath(int blockSize, int numBlocks, Path path) {
HFileBlockPair[] returnedBlocks = new HFileBlockPair[numBlocks];
Random rand = ThreadLocalRandom.current();
HashSet<String> usedStrings = new HashSet<>();
Expand All @@ -299,16 +304,20 @@ public static HFileBlockPair[] generateHFileBlocks(int blockSize, int numBlocks)
prevBlockOffset, ByteBuff.wrap(cachedBuffer), HFileBlock.DONT_FILL_HEADER, blockSize,
onDiskSizeWithoutHeader + HConstants.HFILEBLOCK_HEADER_SIZE, -1, meta,
ByteBuffAllocator.HEAP);

String strKey;
/* No conflicting keys */
strKey = Long.toString(rand.nextLong());
while (!usedStrings.add(strKey)) {
strKey = Long.toString(rand.nextLong());
String key = null;
long offset = 0;
if (path != null) {
key = path.getName();
offset = i * blockSize;
} else {
/* No conflicting keys */
key = Long.toString(rand.nextLong());
while (!usedStrings.add(key)) {
key = Long.toString(rand.nextLong());
}
}

returnedBlocks[i] = new HFileBlockPair();
returnedBlocks[i].blockName = new BlockCacheKey(strKey, 0);
returnedBlocks[i].blockName = new BlockCacheKey(key, offset);
returnedBlocks[i].block = generated;
}
return returnedBlocks;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -890,4 +890,57 @@ public void testBlockAdditionWaitWhenCache() throws Exception {
HBASE_TESTING_UTILITY.cleanupTestDir();
}
}

@Test
public void testNotifyFileCachingCompletedSuccess() throws Exception {
BucketCache bucketCache = null;
try {
Path filePath =
new Path(HBASE_TESTING_UTILITY.getDataTestDir(), "testNotifyFileCachingCompletedSuccess");
bucketCache = testNotifyFileCachingCompleted(filePath, 10);
assertTrue(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
} finally {
if (bucketCache != null) {
bucketCache.shutdown();
}
HBASE_TESTING_UTILITY.cleanupTestDir();
}
}

@Test
public void testNotifyFileCachingCompletedNotAllCached() throws Exception {
BucketCache bucketCache = null;
try {
Path filePath = new Path(HBASE_TESTING_UTILITY.getDataTestDir(),
"testNotifyFileCachingCompletedNotAllCached");
// Deliberately passing more blocks than we have created to test that
// notifyFileCachingCompleted will not consider the file fully cached
bucketCache = testNotifyFileCachingCompleted(filePath, 12);
assertFalse(bucketCache.fullyCachedFiles.containsKey(filePath.getName()));
} finally {
if (bucketCache != null) {
bucketCache.shutdown();
}
HBASE_TESTING_UTILITY.cleanupTestDir();
}
}

private BucketCache testNotifyFileCachingCompleted(Path filePath, int totalBlocks)
throws Exception {
final Path dataTestDir = createAndGetTestDir();
String ioEngineName = "file:" + dataTestDir + "/bucketNoRecycler.cache";
BucketCache bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
constructedBlockSizes, 1, 1, null);
long usedByteSize = bucketCache.getAllocator().getUsedSize();
assertEquals(0, usedByteSize);
HFileBlockPair[] hfileBlockPairs =
CacheTestUtils.generateBlocksForPath(constructedBlockSize, 10, filePath);
// Add blocks
for (HFileBlockPair hfileBlockPair : hfileBlockPairs) {
bucketCache.cacheBlock(hfileBlockPair.getBlockName(), hfileBlockPair.getBlock(), false, true);
}
bucketCache.notifyFileCachingCompleted(filePath, totalBlocks, totalBlocks,
totalBlocks * constructedBlockSize);
return bucketCache;
}
}