Skip to content

Commit 2f6b6aa

Browse files
authored
HBASE-28324 TestRegionNormalizerWorkQueue#testTake is flaky (#5643)
Signed-off-by: Duo Zhang <[email protected]>
1 parent fdde227 commit 2f6b6aa

1 file changed

Lines changed: 7 additions & 6 deletions

File tree

hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestRegionNormalizerWorkQueue.java

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import static org.hamcrest.Matchers.greaterThan;
2323
import static org.hamcrest.Matchers.lessThanOrEqualTo;
2424
import static org.junit.Assert.assertEquals;
25-
import static org.junit.Assert.assertTrue;
2625
import static org.junit.Assert.fail;
2726

2827
import java.util.ArrayList;
@@ -41,6 +40,8 @@
4140
import java.util.stream.Collectors;
4241
import java.util.stream.IntStream;
4342
import org.apache.hadoop.hbase.HBaseClassTestRule;
43+
import org.apache.hadoop.hbase.HBaseConfiguration;
44+
import org.apache.hadoop.hbase.Waiter;
4445
import org.apache.hadoop.hbase.testclassification.MasterTests;
4546
import org.apache.hadoop.hbase.testclassification.SmallTests;
4647
import org.junit.ClassRule;
@@ -186,6 +187,7 @@ public void testTake() throws Exception {
186187
final RegionNormalizerWorkQueue<Integer> queue = new RegionNormalizerWorkQueue<>();
187188
final ConcurrentLinkedQueue<Long> takeTimes = new ConcurrentLinkedQueue<>();
188189
final AtomicBoolean finished = new AtomicBoolean(false);
190+
final int count = 5;
189191
final Runnable consumer = () -> {
190192
try {
191193
while (!finished.get()) {
@@ -199,21 +201,20 @@ public void testTake() throws Exception {
199201

200202
CompletableFuture<Void> worker = CompletableFuture.runAsync(consumer);
201203
final long testStart = System.nanoTime();
202-
for (int i = 0; i < 5; i++) {
204+
for (int i = 0; i < count; i++) {
203205
Thread.sleep(10);
204206
queue.put(i);
205207
}
206-
208+
// should have timing information for 5 calls to take.
209+
Waiter.waitFor(HBaseConfiguration.create(), 1000, () -> count == takeTimes.size());
207210
// set finished = true and pipe one more value in case the thread needs an extra pass through
208211
// the loop.
209212
finished.set(true);
210213
queue.put(1);
211214
worker.get(1, TimeUnit.SECONDS);
212215

213216
final Iterator<Long> times = takeTimes.iterator();
214-
assertTrue("should have timing information for at least 2 calls to take.",
215-
takeTimes.size() >= 5);
216-
for (int i = 0; i < 5; i++) {
217+
for (int i = 0; i < count; i++) {
217218
assertThat(
218219
"Observations collected in takeTimes should increase by roughly 10ms every interval",
219220
times.next(), greaterThan(testStart + TimeUnit.MILLISECONDS.toNanos(i * 10)));

0 commit comments

Comments
 (0)