diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 661874214a85..4bbfc6597696 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -239,6 +239,7 @@ private HStore init(String methodName, Configuration conf, TableDescriptorBuilde } else { store = new MyStore(region, hcd, conf, hook, switchToPread); } + region.stores.put(store.getColumnFamilyDescriptor().getName(), store); return store; } @@ -1791,14 +1792,16 @@ public void testCompactingMemStoreNoCellButDataSizeExceedsInmemoryFlushSize() // 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 void testCompactingMemStoreCellExceedInmemoryFlushSize() 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 @@ -2795,4 +2800,28 @@ void inMemoryCompaction() { } } } + + 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); + } + + } + } + } }