Skip to content

Commit da0a694

Browse files
committed
HBASE-28804: Addressing first round of review comments.
Change-Id: Ie160b249f6a2bff18fd8a577ae32a263b7de25ea
1 parent 8377eb9 commit da0a694

3 files changed

Lines changed: 21 additions & 45 deletions

File tree

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

Lines changed: 3 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@
5555
import org.apache.hadoop.hbase.KeyValue;
5656
import org.apache.hadoop.hbase.MatcherPredicate;
5757
import org.apache.hadoop.hbase.TableName;
58+
import org.apache.hadoop.hbase.Waiter;
5859
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
5960
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
6061
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -367,16 +368,10 @@ public void testPrefetchWithDelay() throws Exception {
367368
Thread.sleep(20000);
368369
assertFalse("Prefetch threads should not be running at this point", reader.prefetchStarted());
369370
long timeout = 10000;
370-
while (!reader.prefetchStarted() && !reader.prefetchComplete()) {
371-
// Wait until the prefetch is triggered.
372-
Thread.sleep(500);
373-
if (timeout <= 0) break;
374-
timeout -= 500;
375-
}
371+
Waiter.waitFor(conf, 10000, () -> (reader.prefetchStarted() || reader.prefetchComplete()));
372+
376373
assertTrue(reader.prefetchStarted() || reader.prefetchComplete());
377374

378-
// Added some delay as we have started the timer a bit late.
379-
Thread.sleep(500);
380375
assertTrue("Prefetch should start post configured delay",
381376
getElapsedTime(startTime) > PrefetchExecutor.getPrefetchDelay());
382377

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

Lines changed: 17 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -226,19 +226,9 @@ public void testHeapSizeChanges() throws Exception {
226226

227227
public static void waitUntilFlushedToBucket(BucketCache cache, BlockCacheKey cacheKey)
228228
throws InterruptedException {
229-
long timeout = 120000;
230-
try {
231-
while (!cache.backingMap.containsKey(cacheKey) || cache.ramCache.containsKey(cacheKey)) {
232-
Thread.sleep(100);
233-
if (timeout <= 0) {
234-
break;
235-
}
236-
timeout -= 100;
237-
}
238-
} finally {
239-
Thread.sleep(1000);
229+
while (!cache.backingMap.containsKey(cacheKey) || cache.ramCache.containsKey(cacheKey)) {
230+
Thread.sleep(100);
240231
}
241-
assertTrue(cache.backingMap.containsKey(cacheKey) && !cache.ramCache.containsKey(cacheKey));
242232
}
243233

244234
public static void waitUntilAllFlushedToBucket(BucketCache cache) throws InterruptedException {
@@ -312,7 +302,7 @@ public void testRetrieveFromFile() throws Exception {
312302
try {
313303
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
314304
smallBucketSizes, writeThreads, writerQLen, persistencePath);
315-
bucketCache.waitForCacheInitialization(10000);
305+
assertTrue(bucketCache.waitForCacheInitialization(10000));
316306
assertFalse(new File(persistencePath).exists());
317307
assertEquals(0, bucketCache.getAllocator().getUsedSize());
318308
assertEquals(0, bucketCache.backingMap.size());
@@ -340,7 +330,7 @@ public void testRetrieveFromPMem() throws Exception {
340330
try {
341331
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
342332
smallBucketSizes, writeThreads, writerQLen, persistencePath);
343-
bucketCache.waitForCacheInitialization(10000);
333+
assertTrue(bucketCache.waitForCacheInitialization(10000));
344334
assertFalse(new File(persistencePath).exists());
345335
assertEquals(0, bucketCache.getAllocator().getUsedSize());
346336
assertEquals(0, bucketCache.backingMap.size());
@@ -358,7 +348,7 @@ private void testRetrievalUtils(Path testDir, String ioEngineName)
358348
try {
359349
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
360350
constructedBlockSizes, writeThreads, writerQLen, persistencePath);
361-
bucketCache.waitForCacheInitialization(10000);
351+
assertTrue(bucketCache.waitForCacheInitialization(10000));
362352
long usedSize = bucketCache.getAllocator().getUsedSize();
363353
assertEquals(0, usedSize);
364354
HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(constructedBlockSize, 1);
@@ -375,7 +365,7 @@ private void testRetrievalUtils(Path testDir, String ioEngineName)
375365
assertTrue(new File(persistencePath).exists());
376366
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
377367
constructedBlockSizes, writeThreads, writerQLen, persistencePath);
378-
bucketCache.waitForCacheInitialization(10000);
368+
assertTrue(bucketCache.waitForCacheInitialization(10000));
379369

380370
assertEquals(usedSize, bucketCache.getAllocator().getUsedSize());
381371
} finally {
@@ -414,7 +404,7 @@ public void testRetrieveFromMultipleFiles() throws Exception {
414404
try {
415405
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
416406
smallBucketSizes, writeThreads, writerQLen, persistencePath);
417-
bucketCache.waitForCacheInitialization(10000);
407+
assertTrue(bucketCache.waitForCacheInitialization(10000));
418408
assertFalse(new File(persistencePath).exists());
419409
assertEquals(0, bucketCache.getAllocator().getUsedSize());
420410
assertEquals(0, bucketCache.backingMap.size());
@@ -428,7 +418,7 @@ public void testRetrieveFromMultipleFiles() throws Exception {
428418
public void testRetrieveFromFileWithoutPersistence() throws Exception {
429419
BucketCache bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
430420
constructedBlockSizes, writeThreads, writerQLen, null);
431-
bucketCache.waitForCacheInitialization(10000);
421+
assertTrue(bucketCache.waitForCacheInitialization(10000));
432422
try {
433423
final Path testDir = createAndGetTestDir();
434424
String ioEngineName = "file:" + testDir + "/bucket.cache";
@@ -447,7 +437,7 @@ public void testRetrieveFromFileWithoutPersistence() throws Exception {
447437
bucketCache.shutdown();
448438
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
449439
constructedBlockSizes, writeThreads, writerQLen, null);
450-
bucketCache.waitForCacheInitialization(10000);
440+
assertTrue(bucketCache.waitForCacheInitialization(10000));
451441
assertEquals(0, bucketCache.getAllocator().getUsedSize());
452442
} finally {
453443
bucketCache.shutdown();
@@ -477,7 +467,7 @@ public void testGetPartitionSize() throws IOException {
477467

478468
BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
479469
constructedBlockSizes, writeThreads, writerQLen, null, 100, conf);
480-
CacheTestUtils.waitForCacheInitialization(cache, 10000);
470+
assertTrue(cache.waitForCacheInitialization(10000));
481471

482472
validateGetPartitionSize(cache, 0.1f, 0.5f);
483473
validateGetPartitionSize(cache, 0.7f, 0.5f);
@@ -515,7 +505,7 @@ public void testValidBucketCacheConfigs() throws IOException {
515505

516506
BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
517507
constructedBlockSizes, writeThreads, writerQLen, null, 100, conf);
518-
CacheTestUtils.waitForCacheInitialization(cache, 10000);
508+
assertTrue(cache.waitForCacheInitialization(10000));
519509

520510
assertEquals(BucketCache.ACCEPT_FACTOR_CONFIG_NAME + " failed to propagate.", 0.9f,
521511
cache.getAcceptableFactor(), 0);
@@ -589,7 +579,7 @@ private void checkConfigValues(Configuration conf, Map<String, float[]> configMa
589579
}
590580
BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
591581
constructedBlockSizes, writeThreads, writerQLen, null, 100, conf);
592-
CacheTestUtils.waitForCacheInitialization(cache, 10000);
582+
assertTrue(cache.waitForCacheInitialization(10000));
593583
assertTrue("Created BucketCache and expected it to succeed: " + expectSuccess[i]
594584
+ ", but it actually was: " + !expectSuccess[i], expectSuccess[i]);
595585
} catch (IllegalArgumentException e) {
@@ -816,7 +806,7 @@ public void testFreeBucketEntryRestoredFromFile() throws Exception {
816806

817807
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
818808
constructedBlockSizes, writeThreads, writerQLen, persistencePath);
819-
CacheTestUtils.waitForCacheInitialization(bucketCache, 10000);
809+
assertTrue(bucketCache.waitForCacheInitialization(10000));
820810
long usedByteSize = bucketCache.getAllocator().getUsedSize();
821811
assertEquals(0, usedByteSize);
822812

@@ -840,7 +830,7 @@ public void testFreeBucketEntryRestoredFromFile() throws Exception {
840830
// restore cache from file
841831
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
842832
constructedBlockSizes, writeThreads, writerQLen, persistencePath);
843-
CacheTestUtils.waitForCacheInitialization(bucketCache, 10000);
833+
assertTrue(bucketCache.waitForCacheInitialization(10000));
844834
assertEquals(usedByteSize, bucketCache.getAllocator().getUsedSize());
845835

846836
for (HFileBlockPair hfileBlockPair : hfileBlockPairs) {
@@ -866,7 +856,7 @@ public void testBlockAdditionWaitWhenCache() throws Exception {
866856

867857
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
868858
constructedBlockSizes, 1, 1, persistencePath);
869-
CacheTestUtils.waitForCacheInitialization(bucketCache, 10000);
859+
assertTrue(bucketCache.waitForCacheInitialization(10000));
870860
long usedByteSize = bucketCache.getAllocator().getUsedSize();
871861
assertEquals(0, usedByteSize);
872862

@@ -898,7 +888,7 @@ public void testBlockAdditionWaitWhenCache() throws Exception {
898888
// restore cache from file
899889
bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
900890
constructedBlockSizes, writeThreads, writerQLen, persistencePath);
901-
CacheTestUtils.waitForCacheInitialization(bucketCache, 10000);
891+
assertTrue(bucketCache.waitForCacheInitialization(10000));
902892
assertEquals(usedByteSize, bucketCache.getAllocator().getUsedSize());
903893

904894
for (HFileBlockPair hfileBlockPair : hfileBlockPairs) {
@@ -955,7 +945,7 @@ private BucketCache testNotifyFileCachingCompleted(Path filePath, int totalBlock
955945
String ioEngineName = "file:" + dataTestDir + "/bucketNoRecycler.cache";
956946
BucketCache bucketCache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
957947
constructedBlockSizes, 1, 1, null);
958-
CacheTestUtils.waitForCacheInitialization(bucketCache, 10000);
948+
assertTrue(bucketCache.waitForCacheInitialization(10000));
959949
long usedByteSize = bucketCache.getAllocator().getUsedSize();
960950
assertEquals(0, usedByteSize);
961951
HFileBlockPair[] hfileBlockPairs =

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

Lines changed: 1 addition & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -147,16 +147,7 @@ public void testBucketCacheEvictByHFileAfterRecovery() throws Exception {
147147

148148
private void waitUntilFlushedToBucket(BucketCache cache, BlockCacheKey cacheKey)
149149
throws InterruptedException {
150-
try {
151-
long timeout = 120000;
152-
while (!cache.backingMap.containsKey(cacheKey) || cache.ramCache.containsKey(cacheKey)) {
153-
Thread.sleep(100);
154-
if (timeout <= 0) {
155-
break;
156-
}
157-
timeout -= 100;
158-
}
159-
} finally {
150+
while (!cache.backingMap.containsKey(cacheKey) || cache.ramCache.containsKey(cacheKey)) {
160151
Thread.sleep(100);
161152
}
162153
}

0 commit comments

Comments
 (0)