HBASE-19930: Fixing the bug, in the rare case when there is a merge into CCM and the one of the old segments has a big cell allocated on-heap and it needs to be copied to the MSLAB
This commit is contained in:
parent
c18e7a963d
commit
07e93458fe
|
@ -45,9 +45,20 @@ public class ImmutableMemStoreLAB implements MemStoreLAB {
|
|||
throw new IllegalStateException("This is an Immutable MemStoreLAB.");
|
||||
}
|
||||
|
||||
/**
|
||||
* The process of merging assumes all cells are allocated on mslab.
|
||||
* There is a rare case in which the first immutable segment,
|
||||
* participating in a merge, is a CSLM.
|
||||
* Since the CSLM hasn't been flattened yet, and there is no point in flattening it (since it is
|
||||
* going to be merged), its big cells (for whom size > maxAlloc) must be copied into mslab.
|
||||
* This method copies the passed cell into the first mslab in the mslabs list,
|
||||
* returning either a new cell instance over the copied data,
|
||||
* or null when this cell cannt be copied.
|
||||
*/
|
||||
@Override
|
||||
public Cell forceCopyOfBigCellInto(Cell cell) {
|
||||
throw new IllegalStateException("This is an Immutable MemStoreLAB.");
|
||||
MemStoreLAB mslab = this.mslabs.get(0);
|
||||
return mslab.forceCopyOfBigCellInto(cell);
|
||||
}
|
||||
|
||||
/* Creating chunk to be used as index chunk in CellChunkMap, part of the chunks array.
|
||||
|
|
|
@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
|
@ -823,6 +824,71 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore
|
|||
assertEquals(totalHeapSize, ((CompactingMemStore) memstore).heapSize());
|
||||
}
|
||||
|
||||
/**
|
||||
* CellChunkMap Segment index requires all cell data to be written in the MSLAB Chunks.
|
||||
* Even though MSLAB is enabled, cells bigger than the size of a chunk are not
|
||||
* written in the MSLAB Chunks.
|
||||
* If such cells are found in the process of a merge they need to be copied into MSLAB.
|
||||
* testForceCopyOfBigCellIntoImmutableSegment checks that the
|
||||
* ImmutableMemStoreLAB's forceCopyOfBigCellInto does what it's supposed to do.
|
||||
*/
|
||||
@Test
|
||||
public void testForceCopyOfBigCellIntoImmutableSegment() throws IOException {
|
||||
|
||||
if (toCellChunkMap == false) {
|
||||
return;
|
||||
}
|
||||
|
||||
// set memstore to flat into CellChunkMap
|
||||
MemoryCompactionPolicy compactionType = MemoryCompactionPolicy.BASIC;
|
||||
memstore.getConfiguration().set(CompactingMemStore.COMPACTING_MEMSTORE_TYPE_KEY,
|
||||
String.valueOf(compactionType));
|
||||
((MyCompactingMemStore) memstore).initiateType(compactionType, memstore.getConfiguration());
|
||||
((CompactingMemStore) memstore).setIndexType(CompactingMemStore.IndexType.CHUNK_MAP);
|
||||
|
||||
char[] chars = new char[MemStoreLAB.CHUNK_SIZE_DEFAULT];
|
||||
for (int i = 0; i < chars.length; i++) {
|
||||
chars[i] = 'A';
|
||||
}
|
||||
String bigVal = new String(chars);
|
||||
byte[] val = Bytes.toBytes(bigVal);
|
||||
|
||||
// We need to add two cells, five times, in order to guarantee a merge
|
||||
List<String[]> keysList = new ArrayList<>();
|
||||
keysList.add(new String[]{"A", "B"});
|
||||
keysList.add(new String[]{"C", "D"});
|
||||
keysList.add(new String[]{"E", "F"});
|
||||
keysList.add(new String[]{"G", "H"});
|
||||
keysList.add(new String[]{"I", "J"});
|
||||
|
||||
// Measuring the size of a single kv
|
||||
KeyValue kv = new KeyValue(Bytes.toBytes("A"), Bytes.toBytes("testfamily"),
|
||||
Bytes.toBytes("testqualifier"), System.currentTimeMillis(), val);
|
||||
long oneCellOnCCMHeapSize =
|
||||
ClassSize.CELL_CHUNK_MAP_ENTRY + ClassSize.align(KeyValueUtil.length(kv));
|
||||
|
||||
long totalHeapSize = MutableSegment.DEEP_OVERHEAD;
|
||||
for (int i = 0; i < 5; i++) {
|
||||
addRowsByKeys(memstore, keysList.get(i), val);
|
||||
while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
|
||||
Threads.sleep(10);
|
||||
}
|
||||
|
||||
// The in-memory flush size is bigger than the size of a single cell,
|
||||
// but smaller than the size of two cells.
|
||||
// Therefore, the two created cells are flattened together.
|
||||
totalHeapSize += CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
||||
+ 2 * oneCellOnCCMHeapSize;
|
||||
if (i == 4) {
|
||||
// Four out of the five are merged into one,
|
||||
// and the segment becomes immutable
|
||||
totalHeapSize -= (3 * CellChunkImmutableSegment.DEEP_OVERHEAD_CCM
|
||||
+ MutableSegment.DEEP_OVERHEAD);
|
||||
}
|
||||
assertEquals(totalHeapSize, ((CompactingMemStore) memstore).heapSize());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private long addRowsByKeysDataSize(final AbstractMemStore hmc, String[] keys) {
|
||||
byte[] fam = Bytes.toBytes("testfamily");
|
||||
|
|
Loading…
Reference in New Issue