HBASE-26438 Fix flaky test TestHStore.testCompactingMemStoreCellExceedInmemoryFlushSize (#3834)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
This commit is contained in:
parent
e1ce414b1b
commit
8458e44a1a
|
@ -239,6 +239,7 @@ public class TestHStore {
|
|||
} else {
|
||||
store = new MyStore(region, hcd, conf, hook, switchToPread);
|
||||
}
|
||||
region.stores.put(store.getColumnFamilyDescriptor().getName(), store);
|
||||
return store;
|
||||
}
|
||||
|
||||
|
@ -1791,14 +1792,16 @@ public class TestHStore {
|
|||
// InmemoryFlushSize
|
||||
@Test(timeout = 60000)
|
||||
public void testCompactingMemStoreCellExceedInmemoryFlushSize()
|
||||
throws IOException, InterruptedException {
|
||||
throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.set(HStore.MEMSTORE_CLASS_NAME, CompactingMemStore.class.getName());
|
||||
conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore6.class.getName());
|
||||
|
||||
init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
|
||||
|
||||
int size = (int) ((CompactingMemStore) store.memstore).getInmemoryFlushSize();
|
||||
MyCompactingMemStore6 myCompactingMemStore = ((MyCompactingMemStore6) store.memstore);
|
||||
|
||||
int size = (int) (myCompactingMemStore.getInmemoryFlushSize());
|
||||
byte[] value = new byte[size + 1];
|
||||
|
||||
MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
|
||||
|
@ -1809,6 +1812,8 @@ public class TestHStore {
|
|||
store.add(cell, memStoreSizing);
|
||||
assertTrue(memStoreSizing.getCellsCount() == 1);
|
||||
assertTrue(memStoreSizing.getDataSize() == cellByteSize);
|
||||
// Waiting the in memory compaction completed, see HBASE-26438
|
||||
myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();
|
||||
}
|
||||
|
||||
// This test is for HBASE-26210 also, test write large cell and small cell concurrently when
|
||||
|
@ -2818,4 +2823,28 @@ public class TestHStore {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class MyCompactingMemStore6 extends CompactingMemStore {
|
||||
private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);
|
||||
|
||||
public MyCompactingMemStore6(Configuration conf, CellComparatorImpl cellComparator,
|
||||
HStore store, RegionServicesForStores regionServices,
|
||||
MemoryCompactionPolicy compactionPolicy) throws IOException {
|
||||
super(conf, cellComparator, store, regionServices, compactionPolicy);
|
||||
}
|
||||
|
||||
@Override
|
||||
void inMemoryCompaction() {
|
||||
try {
|
||||
super.inMemoryCompaction();
|
||||
} finally {
|
||||
try {
|
||||
inMemoryCompactionEndCyclicBarrier.await();
|
||||
} catch (Throwable e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue