HBASE-16747 Track memstore data size and heap overhead separately.

This commit is contained in:
anoopsamjohn 2016-10-30 12:20:46 +05:30
parent 6127753b65
commit ba6d952324
44 changed files with 1218 additions and 1347 deletions

View File

@ -381,6 +381,7 @@ public final class CellUtil {
private static class TagRewriteCell implements ExtendedCell { private static class TagRewriteCell implements ExtendedCell {
protected Cell cell; protected Cell cell;
protected byte[] tags; protected byte[] tags;
private static final long HEAP_SIZE_OVERHEAD = 2 * ClassSize.REFERENCE + ClassSize.ARRAY;
/** /**
* @param cell The original Cell which it rewrites * @param cell The original Cell which it rewrites
@ -552,6 +553,11 @@ public final class CellUtil {
offset = Bytes.putAsShort(buf, offset, tagsLen); offset = Bytes.putAsShort(buf, offset, tagsLen);
System.arraycopy(this.tags, 0, buf, offset, tagsLen); System.arraycopy(this.tags, 0, buf, offset, tagsLen);
} }
@Override
public long heapOverhead() {
return ((ExtendedCell) this.cell).heapOverhead() + HEAP_SIZE_OVERHEAD;
}
} }
/** /**

View File

@ -66,4 +66,9 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
* @param offset The offset within buffer, to write the Cell. * @param offset The offset within buffer, to write the Cell.
*/ */
void write(byte[] buf, int offset); void write(byte[] buf, int offset);
/**
* @return The heap size overhead associated with this Cell.
*/
long heapOverhead();
} }

View File

@ -85,6 +85,11 @@ public class KeyValue implements ExtendedCell {
private static final Log LOG = LogFactory.getLog(KeyValue.class); private static final Log LOG = LogFactory.getLog(KeyValue.class);
public static final long FIXED_OVERHEAD = ClassSize.OBJECT + // the KeyValue object itself
ClassSize.REFERENCE + // pointer to "bytes"
2 * Bytes.SIZEOF_INT + // offset, length
Bytes.SIZEOF_LONG;// memstoreTS
/** /**
* Colon character in UTF-8 * Colon character in UTF-8
*/ */
@ -2603,12 +2608,7 @@ public class KeyValue implements ExtendedCell {
*/ */
@Override @Override
public long heapSize() { public long heapSize() {
int sum = 0; long sum = FIXED_OVERHEAD;
sum += ClassSize.OBJECT;// the KeyValue object itself
sum += ClassSize.REFERENCE;// pointer to "bytes"
sum += 2 * Bytes.SIZEOF_INT;// offset, length
sum += Bytes.SIZEOF_LONG;// memstoreTS
/* /*
* Deep object overhead for this KV consists of two parts. The first part is the KV object * Deep object overhead for this KV consists of two parts. The first part is the KV object
* itself, while the second part is the backing byte[]. We will only count the array overhead * itself, while the second part is the backing byte[]. We will only count the array overhead
@ -2812,5 +2812,15 @@ public class KeyValue implements ExtendedCell {
// of Cell to be returned back over the RPC // of Cell to be returned back over the RPC
throw new IllegalStateException("A reader should never return this type of a Cell"); throw new IllegalStateException("A reader should never return this type of a Cell");
} }
@Override
public long heapOverhead() {
return super.heapOverhead() + Bytes.SIZEOF_SHORT;
}
}
@Override
public long heapOverhead() {
return FIXED_OVERHEAD;
} }
} }

View File

@ -42,7 +42,7 @@ public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
private final boolean hasTags; private final boolean hasTags;
// TODO : See if famLen can be cached or not? // TODO : See if famLen can be cached or not?
private static final int FIXED_HEAP_SIZE_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE private static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE
+ (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT + (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_SHORT
+ Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG; + Bytes.SIZEOF_BOOLEAN + Bytes.SIZEOF_LONG;
@ -235,7 +235,7 @@ public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
@Override @Override
public long heapSize() { public long heapSize() {
return ClassSize.align(FIXED_HEAP_SIZE_OVERHEAD + ClassSize.align(length)); return ClassSize.align(FIXED_OVERHEAD + ClassSize.align(length));
} }
@Override @Override
@ -276,4 +276,9 @@ public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
// TODO when doing HBASE-15179 // TODO when doing HBASE-15179
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
public long heapOverhead() {
return FIXED_OVERHEAD;
}
} }

View File

@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private @InterfaceAudience.Private
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS") @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS")
public class SizeCachedKeyValue extends KeyValue { public class SizeCachedKeyValue extends KeyValue {
private static final int HEAP_SIZE_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT; // Overhead in this class alone. Parent's overhead will be considered in usage places by calls to
// super. methods
private static final int FIXED_OVERHEAD = Bytes.SIZEOF_SHORT + Bytes.SIZEOF_INT;
private short rowLen; private short rowLen;
private int keyLen; private int keyLen;
@ -58,6 +60,11 @@ public class SizeCachedKeyValue extends KeyValue {
@Override @Override
public long heapSize() { public long heapSize() {
return super.heapSize() + HEAP_SIZE_OVERHEAD; return super.heapSize() + FIXED_OVERHEAD;
}
@Override
public long heapOverhead() {
return super.heapOverhead() + FIXED_OVERHEAD;
} }
} }

View File

@ -468,6 +468,11 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
// This is not used in actual flow. Throwing UnsupportedOperationException // This is not used in actual flow. Throwing UnsupportedOperationException
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
public long heapOverhead() {
return FIXED_OVERHEAD;
}
} }
protected static class OffheapDecodedCell extends ByteBufferedCell implements ExtendedCell { protected static class OffheapDecodedCell extends ByteBufferedCell implements ExtendedCell {
@ -707,6 +712,11 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
// This is not used in actual flow. Throwing UnsupportedOperationException // This is not used in actual flow. Throwing UnsupportedOperationException
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
public long heapOverhead() {
return FIXED_OVERHEAD;
}
} }
protected abstract static class BufferedEncodedSeeker<STATE extends SeekerState> protected abstract static class BufferedEncodedSeeker<STATE extends SeekerState>

View File

@ -129,7 +129,7 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
scanner.close(); scanner.close();
} }
LOG.info("Mob store is flushed, sequenceid=" + cacheFlushId + ", memsize=" LOG.info("Mob store is flushed, sequenceid=" + cacheFlushId + ", memsize="
+ StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getSize(), "", 1) + + StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getDataSize(), "", 1) +
", hasBloomFilter=" + writer.hasGeneralBloom() + ", hasBloomFilter=" + writer.hasGeneralBloom() +
", into tmp file " + writer.getPath()); ", into tmp file " + writer.getPath());
result.add(writer.getPath()); result.add(writer.getPath());

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.SortedSet; import java.util.SortedSet;
@ -30,9 +28,6 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.ShareableMemory; import org.apache.hadoop.hbase.ShareableMemory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
@ -78,18 +73,6 @@ public abstract class AbstractMemStore implements MemStore {
this.timeOfOldestEdit = Long.MAX_VALUE; this.timeOfOldestEdit = Long.MAX_VALUE;
} }
/*
* Calculate how the MemStore size has changed. Includes overhead of the
* backing Map.
* @param cell
* @param notPresent True if the cell was NOT present in the set.
* @return change in size
*/
static long heapSizeChange(final Cell cell, final boolean notPresent) {
return notPresent ? ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY
+ CellUtil.estimatedHeapSizeOf(cell)) : 0;
}
/** /**
* Updates the wal with the lowest sequence id (oldest entry) that is still in memory * Updates the wal with the lowest sequence id (oldest entry) that is still in memory
* @param onlyIfMoreRecent a flag that marks whether to update the sequence id no matter what or * @param onlyIfMoreRecent a flag that marks whether to update the sequence id no matter what or
@ -98,22 +81,14 @@ public abstract class AbstractMemStore implements MemStore {
public abstract void updateLowestUnflushedSequenceIdInWAL(boolean onlyIfMoreRecent); public abstract void updateLowestUnflushedSequenceIdInWAL(boolean onlyIfMoreRecent);
@Override @Override
public long add(Iterable<Cell> cells) { public void add(Iterable<Cell> cells, MemstoreSize memstoreSize) {
long size = 0;
for (Cell cell : cells) { for (Cell cell : cells) {
size += add(cell); add(cell, memstoreSize);
} }
return size;
} }
/**
* Write an update
* @param cell the cell to be added
* @return approximate size of the passed cell & newly added cell which maybe different than the
* passed-in cell
*/
@Override @Override
public long add(Cell cell) { public void add(Cell cell, MemstoreSize memstoreSize) {
Cell toAdd = maybeCloneWithAllocator(cell); Cell toAdd = maybeCloneWithAllocator(cell);
boolean mslabUsed = (toAdd != cell); boolean mslabUsed = (toAdd != cell);
// This cell data is backed by the same byte[] where we read request in RPC(See HBASE-15180). By // This cell data is backed by the same byte[] where we read request in RPC(See HBASE-15180). By
@ -128,7 +103,7 @@ public abstract class AbstractMemStore implements MemStore {
if (!mslabUsed) { if (!mslabUsed) {
toAdd = deepCopyIfNeeded(toAdd); toAdd = deepCopyIfNeeded(toAdd);
} }
return internalAdd(toAdd, mslabUsed); internalAdd(toAdd, mslabUsed, memstoreSize);
} }
private static Cell deepCopyIfNeeded(Cell cell) { private static Cell deepCopyIfNeeded(Cell cell) {
@ -141,31 +116,11 @@ public abstract class AbstractMemStore implements MemStore {
return cell; return cell;
} }
/**
* Update or insert the specified Cells.
* <p>
* For each Cell, insert into MemStore. This will atomically upsert the
* value for that row/family/qualifier. If a Cell did already exist,
* it will then be removed.
* <p>
* Currently the memstoreTS is kept at 0 so as each insert happens, it will
* be immediately visible. May want to change this so it is atomic across
* all Cells.
* <p>
* This is called under row lock, so Get operations will still see updates
* atomically. Scans will only see each Cell update as atomic.
*
* @param cells the cells to be updated
* @param readpoint readpoint below which we can safely remove duplicate KVs
* @return change in memstore size
*/
@Override @Override
public long upsert(Iterable<Cell> cells, long readpoint) { public void upsert(Iterable<Cell> cells, long readpoint, MemstoreSize memstoreSize) {
long size = 0;
for (Cell cell : cells) { for (Cell cell : cells) {
size += upsert(cell, readpoint); upsert(cell, readpoint, memstoreSize);
} }
return size;
} }
/** /**
@ -176,18 +131,6 @@ public abstract class AbstractMemStore implements MemStore {
return timeOfOldestEdit; return timeOfOldestEdit;
} }
/**
* Write a delete
* @param deleteCell the cell to be deleted
* @return approximate size of the passed key and value.
*/
@Override
public long delete(Cell deleteCell) {
// Delete operation just adds the delete marker cell coming here.
return add(deleteCell);
}
/** /**
* The passed snapshot was successfully persisted; it can be let go. * The passed snapshot was successfully persisted; it can be let go.
* @param id Id of the snapshot to clean out. * @param id Id of the snapshot to clean out.
@ -210,18 +153,9 @@ public abstract class AbstractMemStore implements MemStore {
oldSnapshot.close(); oldSnapshot.close();
} }
/**
* Get the entire heap usage for this MemStore not including keys in the
* snapshot.
*/
@Override @Override
public long heapSize() { public MemstoreSize getSnapshotSize() {
return size(); return new MemstoreSize(this.snapshot.keySize(), this.snapshot.heapOverhead());
}
@Override
public long getSnapshotSize() {
return this.snapshot.keySize();
} }
@Override @Override
@ -249,7 +183,7 @@ public abstract class AbstractMemStore implements MemStore {
} }
/** /*
* Inserts the specified Cell into MemStore and deletes any existing * Inserts the specified Cell into MemStore and deletes any existing
* versions of the same row/family/qualifier as the specified Cell. * versions of the same row/family/qualifier as the specified Cell.
* <p> * <p>
@ -262,9 +196,9 @@ public abstract class AbstractMemStore implements MemStore {
* *
* @param cell the cell to be updated * @param cell the cell to be updated
* @param readpoint readpoint below which we can safely remove duplicate KVs * @param readpoint readpoint below which we can safely remove duplicate KVs
* @return change in size of MemStore * @param memstoreSize
*/ */
private long upsert(Cell cell, long readpoint) { private void upsert(Cell cell, long readpoint, MemstoreSize memstoreSize) {
// Add the Cell to the MemStore // Add the Cell to the MemStore
// Use the internalAdd method here since we (a) already have a lock // Use the internalAdd method here since we (a) already have a lock
// and (b) cannot safely use the MSLAB here without potentially // and (b) cannot safely use the MSLAB here without potentially
@ -275,50 +209,9 @@ public abstract class AbstractMemStore implements MemStore {
// must do below deep copy. Or else we will keep referring to the bigger chunk of memory and // must do below deep copy. Or else we will keep referring to the bigger chunk of memory and
// prevent it from getting GCed. // prevent it from getting GCed.
cell = deepCopyIfNeeded(cell); cell = deepCopyIfNeeded(cell);
long addedSize = internalAdd(cell, false); this.active.upsert(cell, readpoint, memstoreSize);
// Get the Cells for the row/family/qualifier regardless of timestamp.
// For this case we want to clean up any other puts
Cell firstCell = CellUtil.createFirstOnRow(
cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
SortedSet<Cell> ss = active.tailSet(firstCell);
Iterator<Cell> it = ss.iterator();
// versions visible to oldest scanner
int versionsVisible = 0;
while (it.hasNext()) {
Cell cur = it.next();
if (cell == cur) {
// ignore the one just put in
continue;
}
// check that this is the row and column we are interested in, otherwise bail
if (CellUtil.matchingRow(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
// only remove Puts that concurrent scanners cannot possibly see
if (cur.getTypeByte() == KeyValue.Type.Put.getCode() &&
cur.getSequenceId() <= readpoint) {
if (versionsVisible >= 1) {
// if we get here we have seen at least one version visible to the oldest scanner,
// which means we can prove that no scanner will see this version
// false means there was a change, so give us the size.
long delta = heapSizeChange(cur, true);
addedSize -= delta;
active.incSize(-delta);
it.remove();
setOldestEditTimeToNow(); setOldestEditTimeToNow();
} else { checkActiveSize();
versionsVisible++;
}
}
} else {
// past the row or column, done
break;
}
}
return addedSize;
} }
/* /*
@ -359,75 +252,23 @@ public abstract class AbstractMemStore implements MemStore {
return result; return result;
} }
/**
* Given the specs of a column, update it, first by inserting a new record,
* then removing the old one. Since there is only 1 KeyValue involved, the memstoreTS
* will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
* store will ensure that the insert/delete each are atomic. A scanner/reader will either
* get the new value, or the old value and all readers will eventually only see the new
* value after the old was removed.
*/
@VisibleForTesting
@Override
public long updateColumnValue(byte[] row, byte[] family, byte[] qualifier,
long newValue, long now) {
Cell firstCell = KeyValueUtil.createFirstOnRow(row, family, qualifier);
// Is there a Cell in 'snapshot' with the same TS? If so, upgrade the timestamp a bit.
Cell snc = snapshot.getFirstAfter(firstCell);
if(snc != null) {
// is there a matching Cell in the snapshot?
if (CellUtil.matchingRow(snc, firstCell) && CellUtil.matchingQualifier(snc, firstCell)) {
if (snc.getTimestamp() == now) {
now += 1;
}
}
}
// logic here: the new ts MUST be at least 'now'. But it could be larger if necessary.
// But the timestamp should also be max(now, mostRecentTsInMemstore)
// so we cant add the new Cell w/o knowing what's there already, but we also
// want to take this chance to delete some cells. So two loops (sad)
SortedSet<Cell> ss = this.active.tailSet(firstCell);
for (Cell cell : ss) {
// if this isnt the row we are interested in, then bail:
if (!CellUtil.matchingColumn(cell, family, qualifier)
|| !CellUtil.matchingRow(cell, firstCell)) {
break; // rows dont match, bail.
}
// if the qualifier matches and it's a put, just RM it out of the active.
if (cell.getTypeByte() == KeyValue.Type.Put.getCode() &&
cell.getTimestamp() > now && CellUtil.matchingQualifier(firstCell, cell)) {
now = cell.getTimestamp();
}
}
// create or update (upsert) a new Cell with
// 'now' and a 0 memstoreTS == immediately visible
List<Cell> cells = new ArrayList<Cell>(1);
cells.add(new KeyValue(row, family, qualifier, now, Bytes.toBytes(newValue)));
return upsert(cells, 1L);
}
private Cell maybeCloneWithAllocator(Cell cell) { private Cell maybeCloneWithAllocator(Cell cell) {
return active.maybeCloneWithAllocator(cell); return active.maybeCloneWithAllocator(cell);
} }
/** /*
* Internal version of add() that doesn't clone Cells with the * Internal version of add() that doesn't clone Cells with the
* allocator, and doesn't take the lock. * allocator, and doesn't take the lock.
* *
* Callers should ensure they already have the read lock taken * Callers should ensure they already have the read lock taken
* @param toAdd the cell to add * @param toAdd the cell to add
* @param mslabUsed whether using MSLAB * @param mslabUsed whether using MSLAB
* @return the heap size change in bytes * @param memstoreSize
*/ */
private long internalAdd(final Cell toAdd, final boolean mslabUsed) { private void internalAdd(final Cell toAdd, final boolean mslabUsed, MemstoreSize memstoreSize) {
long s = active.add(toAdd, mslabUsed); active.add(toAdd, mslabUsed, memstoreSize);
setOldestEditTimeToNow(); setOldestEditTimeToNow();
checkActiveSize(); checkActiveSize();
return s;
} }
private void setOldestEditTimeToNow() { private void setOldestEditTimeToNow() {
@ -437,11 +278,15 @@ public abstract class AbstractMemStore implements MemStore {
} }
/** /**
* @return The size of the active segment. Means sum of all cell's size. * @return The total size of cells in this memstore. We will not consider cells in the snapshot
*/ */
protected long keySize() { protected abstract long keySize();
return this.active.keySize();
} /**
* @return The total heap overhead of cells in this memstore. We will not consider cells in the
* snapshot
*/
protected abstract long heapOverhead();
protected CellComparator getComparator() { protected CellComparator getComparator() {
return comparator; return comparator;

View File

@ -100,19 +100,19 @@ public class CompactingMemStore extends AbstractMemStore {
} }
/** /**
* @return Total memory occupied by this MemStore. This includes active segment size and heap size * @return Total memory occupied by this MemStore. This won't include any size occupied by the
* overhead of this memstore but won't include any size occupied by the snapshot. We * snapshot. We assume the snapshot will get cleared soon. This is not thread safe and
* assume the snapshot will get cleared soon. This is not thread safe and the memstore may * the memstore may be changed while computing its size. It is the responsibility of the
* be changed while computing its size. It is the responsibility of the caller to make * caller to make sure this doesn't happen.
* sure this doesn't happen.
*/ */
@Override @Override
public long size() { public MemstoreSize size() {
long res = DEEP_OVERHEAD + this.active.size(); MemstoreSize memstoreSize = new MemstoreSize();
memstoreSize.incMemstoreSize(this.active.keySize(), this.active.heapOverhead());
for (Segment item : pipeline.getSegments()) { for (Segment item : pipeline.getSegments()) {
res += CompactionPipeline.ENTRY_OVERHEAD + item.size(); memstoreSize.incMemstoreSize(item.keySize(), item.heapOverhead());
} }
return res; return memstoreSize;
} }
/** /**
@ -163,13 +163,34 @@ public class CompactingMemStore extends AbstractMemStore {
* @return size of data that is going to be flushed * @return size of data that is going to be flushed
*/ */
@Override @Override
public long getFlushableSize() { public MemstoreSize getFlushableSize() {
long snapshotSize = getSnapshotSize(); MemstoreSize snapshotSize = getSnapshotSize();
if (snapshotSize == 0) { if (snapshotSize.getDataSize() == 0) {
// if snapshot is empty the tail of the pipeline is flushed // if snapshot is empty the tail of the pipeline is flushed
snapshotSize = pipeline.getTailSize(); snapshotSize = pipeline.getTailSize();
} }
return snapshotSize > 0 ? snapshotSize : keySize(); return snapshotSize.getDataSize() > 0 ? snapshotSize
: new MemstoreSize(this.active.keySize(), this.active.heapOverhead());
}
@Override
protected long keySize() {
// Need to consider keySize of all segments in pipeline and active
long k = this.active.keySize();
for (Segment segment : this.pipeline.getSegments()) {
k += segment.keySize();
}
return k;
}
@Override
protected long heapOverhead() {
// Need to consider heapOverhead of all segments in pipeline and active
long h = this.active.heapOverhead();
for (Segment segment : this.pipeline.getSegments()) {
h += segment.heapOverhead();
}
return h;
} }
@Override @Override
@ -318,7 +339,7 @@ public class CompactingMemStore extends AbstractMemStore {
} }
private boolean shouldFlushInMemory() { private boolean shouldFlushInMemory() {
if (this.active.size() > inmemoryFlushSize) { // size above flush threshold if (this.active.keySize() > inmemoryFlushSize) { // size above flush threshold
// the inMemoryFlushInProgress is CASed to be true here in order to mutual exclude // the inMemoryFlushInProgress is CASed to be true here in order to mutual exclude
// the insert of the active into the compaction pipeline // the insert of the active into the compaction pipeline
return (inMemoryFlushInProgress.compareAndSet(false,true)); return (inMemoryFlushInProgress.compareAndSet(false,true));
@ -419,7 +440,7 @@ public class CompactingMemStore extends AbstractMemStore {
// debug method // debug method
public void debug() { public void debug() {
String msg = "active size=" + this.active.size(); String msg = "active size=" + this.active.keySize();
msg += " threshold="+IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT* inmemoryFlushSize; msg += " threshold="+IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT* inmemoryFlushSize;
msg += " allow compaction is "+ (allowCompaction.get() ? "true" : "false"); msg += " allow compaction is "+ (allowCompaction.get() ? "true" : "false");
msg += " inMemoryFlushInProgress is "+ (inMemoryFlushInProgress.get() ? "true" : "false"); msg += " inMemoryFlushInProgress is "+ (inMemoryFlushInProgress.get() ? "true" : "false");

View File

@ -115,19 +115,30 @@ public class CompactionPipeline {
} }
if (region != null) { if (region != null) {
// update the global memstore size counter // update the global memstore size counter
long suffixSize = getSegmentsKeySize(suffix); long suffixDataSize = getSegmentsKeySize(suffix);
long newSize = segment.keySize(); long newDataSize = segment.keySize();
long delta = suffixSize - newSize; long dataSizeDelta = suffixDataSize - newDataSize;
assert ( closeSuffix || delta>0 ); // sanity check long suffixHeapOverhead = getSegmentsHeapOverhead(suffix);
long globalMemstoreSize = region.addAndGetGlobalMemstoreSize(-delta); long newHeapOverhead = segment.heapOverhead();
long heapOverheadDelta = suffixHeapOverhead - newHeapOverhead;
region.addMemstoreSize(new MemstoreSize(-dataSizeDelta, -heapOverheadDelta));
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Suffix size: " + suffixSize + " compacted item size: " + newSize LOG.debug("Suffix data size: " + suffixDataSize + " compacted item data size: "
+ " globalMemstoreSize: " + globalMemstoreSize); + newDataSize + ". Suffix heap overhead: " + suffixHeapOverhead
+ " compacted item heap overhead: " + newHeapOverhead);
} }
} }
return true; return true;
} }
private static long getSegmentsHeapOverhead(List<? extends Segment> list) {
long res = 0;
for (Segment segment : list) {
res += segment.heapOverhead();
}
return res;
}
private static long getSegmentsKeySize(List<? extends Segment> list) { private static long getSegmentsKeySize(List<? extends Segment> list) {
long res = 0; long res = 0;
for (Segment segment : list) { for (Segment segment : list) {
@ -160,16 +171,12 @@ public class CompactionPipeline {
for (ImmutableSegment s : pipeline) { for (ImmutableSegment s : pipeline) {
// remember the old size in case this segment is going to be flatten // remember the old size in case this segment is going to be flatten
long sizeBeforeFlat = s.keySize(); MemstoreSize memstoreSize = new MemstoreSize();
long globalMemstoreSize = 0; if (s.flatten(memstoreSize)) {
if (s.flatten()) {
if(region != null) { if(region != null) {
long sizeAfterFlat = s.keySize(); region.addMemstoreSize(memstoreSize);
long delta = sizeBeforeFlat - sizeAfterFlat;
globalMemstoreSize = region.addAndGetGlobalMemstoreSize(-delta);
} }
LOG.debug("Compaction pipeline segment " + s + " was flattened; globalMemstoreSize: " LOG.debug("Compaction pipeline segment " + s + " was flattened");
+ globalMemstoreSize);
return true; return true;
} }
} }
@ -203,9 +210,9 @@ public class CompactionPipeline {
return minSequenceId; return minSequenceId;
} }
public long getTailSize() { public MemstoreSize getTailSize() {
if (isEmpty()) return 0; if (isEmpty()) return MemstoreSize.EMPTY_SIZE;
return pipeline.peekLast().keySize(); return new MemstoreSize(pipeline.peekLast().keySize(), pipeline.peekLast().heapOverhead());
} }
private void swapSuffix(List<ImmutableSegment> suffix, ImmutableSegment segment, private void swapSuffix(List<ImmutableSegment> suffix, ImmutableSegment segment,

View File

@ -104,9 +104,20 @@ public class DefaultMemStore extends AbstractMemStore {
* @return size of data that is going to be flushed from active set * @return size of data that is going to be flushed from active set
*/ */
@Override @Override
public long getFlushableSize() { public MemstoreSize getFlushableSize() {
long snapshotSize = getSnapshotSize(); MemstoreSize snapshotSize = getSnapshotSize();
return snapshotSize > 0 ? snapshotSize : keySize(); return snapshotSize.getDataSize() > 0 ? snapshotSize
: new MemstoreSize(keySize(), heapOverhead());
}
@Override
protected long keySize() {
return this.active.keySize();
}
@Override
protected long heapOverhead() {
return this.active.heapOverhead();
} }
@Override @Override
@ -144,8 +155,8 @@ public class DefaultMemStore extends AbstractMemStore {
} }
@Override @Override
public long size() { public MemstoreSize size() {
return this.active.size() + DEEP_OVERHEAD; return new MemstoreSize(this.active.keySize(), this.active.heapOverhead());
} }
/** /**
@ -179,26 +190,30 @@ public class DefaultMemStore extends AbstractMemStore {
LOG.info("vmInputArguments=" + runtime.getInputArguments()); LOG.info("vmInputArguments=" + runtime.getInputArguments());
DefaultMemStore memstore1 = new DefaultMemStore(); DefaultMemStore memstore1 = new DefaultMemStore();
// TODO: x32 vs x64 // TODO: x32 vs x64
long size = 0;
final int count = 10000; final int count = 10000;
byte [] fam = Bytes.toBytes("col"); byte [] fam = Bytes.toBytes("col");
byte [] qf = Bytes.toBytes("umn"); byte [] qf = Bytes.toBytes("umn");
byte [] empty = new byte[0]; byte [] empty = new byte[0];
MemstoreSize memstoreSize = new MemstoreSize();
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
// Give each its own ts // Give each its own ts
size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty)); memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memstoreSize);
} }
LOG.info("memstore1 estimated size=" + size); LOG.info("memstore1 estimated size="
+ (memstoreSize.getDataSize() + memstoreSize.getHeapOverhead()));
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
size += memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty)); memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memstoreSize);
} }
LOG.info("memstore1 estimated size (2nd loading of same data)=" + size); LOG.info("memstore1 estimated size (2nd loading of same data)="
+ (memstoreSize.getDataSize() + memstoreSize.getHeapOverhead()));
// Make a variably sized memstore. // Make a variably sized memstore.
DefaultMemStore memstore2 = new DefaultMemStore(); DefaultMemStore memstore2 = new DefaultMemStore();
memstoreSize = new MemstoreSize();
for (int i = 0; i < count; i++) { for (int i = 0; i < count; i++) {
size += memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, new byte[i])); memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, new byte[i]), memstoreSize);
} }
LOG.info("memstore2 estimated size=" + size); LOG.info("memstore2 estimated size="
+ (memstoreSize.getDataSize() + memstoreSize.getHeapOverhead()));
final int seconds = 30; final int seconds = 30;
LOG.info("Waiting " + seconds + " seconds while heap dump is taken"); LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
LOG.info("Exiting."); LOG.info("Exiting.");

View File

@ -89,7 +89,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
scanner.close(); scanner.close();
} }
LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize=" LOG.info("Flushed, sequenceid=" + cacheFlushId +", memsize="
+ StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getSize(), "", 1) + + StringUtils.TraditionalBinaryPrefix.long2String(snapshot.getDataSize(), "", 1) +
", hasBloomFilter=" + writer.hasGeneralBloom() + ", hasBloomFilter=" + writer.hasGeneralBloom() +
", into tmp file " + writer.getPath()); ", into tmp file " + writer.getPath());
result.add(writer.getPath()); result.add(writer.getPath());

View File

@ -78,11 +78,12 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
} }
protected boolean shouldFlush(Store store) { protected boolean shouldFlush(Store store) {
if (store.getMemStoreSize() > this.flushSizeLowerBound) { if (store.getSizeOfMemStore().getDataSize() > this.flushSizeLowerBound) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " + LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +
region.getRegionInfo().getEncodedName() + " because memstoreSize=" + region.getRegionInfo().getEncodedName() + " because memstoreSize=" +
store.getMemStoreSize() + " > lower bound=" + this.flushSizeLowerBound); store.getSizeOfMemStore().getDataSize() + " > lower bound="
+ this.flushSizeLowerBound);
} }
return true; return true;
} }

View File

@ -259,7 +259,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// TODO: account for each registered handler in HeapSize computation // TODO: account for each registered handler in HeapSize computation
private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap(); private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap();
private final AtomicLong memstoreSize = new AtomicLong(0); private final AtomicLong memstoreDataSize = new AtomicLong(0);// Track data size in all memstores
private final RegionServicesForStores regionServicesForStores = new RegionServicesForStores(this); private final RegionServicesForStores regionServicesForStores = new RegionServicesForStores(this);
// Debug possible data loss due to WAL off // Debug possible data loss due to WAL off
@ -506,23 +506,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
final FlushResult result; // indicating a failure result from prepare final FlushResult result; // indicating a failure result from prepare
final TreeMap<byte[], StoreFlushContext> storeFlushCtxs; final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
final TreeMap<byte[], List<Path>> committedFiles; final TreeMap<byte[], List<Path>> committedFiles;
final TreeMap<byte[], Long> storeFlushableSize; final TreeMap<byte[], MemstoreSize> storeFlushableSize;
final long startTime; final long startTime;
final long flushOpSeqId; final long flushOpSeqId;
final long flushedSeqId; final long flushedSeqId;
final long totalFlushableSize; final MemstoreSize totalFlushableSize;
/** Constructs an early exit case */ /** Constructs an early exit case */
PrepareFlushResult(FlushResult result, long flushSeqId) { PrepareFlushResult(FlushResult result, long flushSeqId) {
this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, 0); this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, new MemstoreSize());
} }
/** Constructs a successful prepare flush result */ /** Constructs a successful prepare flush result */
PrepareFlushResult( PrepareFlushResult(
TreeMap<byte[], StoreFlushContext> storeFlushCtxs, TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
TreeMap<byte[], List<Path>> committedFiles, TreeMap<byte[], List<Path>> committedFiles,
TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId, TreeMap<byte[], MemstoreSize> storeFlushableSize, long startTime, long flushSeqId,
long flushedSeqId, long totalFlushableSize) { long flushedSeqId, MemstoreSize totalFlushableSize) {
this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime, this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
flushSeqId, flushedSeqId, totalFlushableSize); flushSeqId, flushedSeqId, totalFlushableSize);
} }
@ -531,8 +531,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
FlushResult result, FlushResult result,
TreeMap<byte[], StoreFlushContext> storeFlushCtxs, TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
TreeMap<byte[], List<Path>> committedFiles, TreeMap<byte[], List<Path>> committedFiles,
TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId, TreeMap<byte[], MemstoreSize> storeFlushableSize, long startTime, long flushSeqId,
long flushedSeqId, long totalFlushableSize) { long flushedSeqId, MemstoreSize totalFlushableSize) {
this.result = result; this.result = result;
this.storeFlushCtxs = storeFlushCtxs; this.storeFlushCtxs = storeFlushCtxs;
this.committedFiles = committedFiles; this.committedFiles = committedFiles;
@ -1125,19 +1125,31 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* store * store
* @return the size of memstore in this region * @return the size of memstore in this region
*/ */
public long addAndGetGlobalMemstoreSize(long memStoreSize) { public long addAndGetMemstoreSize(MemstoreSize memstoreSize) {
if (this.rsAccounting != null) { if (this.rsAccounting != null) {
rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize); rsAccounting.incGlobalMemstoreSize(memstoreSize);
} }
long size = this.memstoreSize.addAndGet(memStoreSize); long size = this.memstoreDataSize.addAndGet(memstoreSize.getDataSize());
checkNegativeMemstoreDataSize(size, memstoreSize.getDataSize());
return size;
}
public void decrMemstoreSize(MemstoreSize memstoreSize) {
if (this.rsAccounting != null) {
rsAccounting.decGlobalMemstoreSize(memstoreSize);
}
long size = this.memstoreDataSize.addAndGet(-memstoreSize.getDataSize());
checkNegativeMemstoreDataSize(size, -memstoreSize.getDataSize());
}
private void checkNegativeMemstoreDataSize(long memstoreDataSize, long delta) {
// This is extremely bad if we make memstoreSize negative. Log as much info on the offending // This is extremely bad if we make memstoreSize negative. Log as much info on the offending
// caller as possible. (memStoreSize might be a negative value already -- freeing memory) // caller as possible. (memStoreSize might be a negative value already -- freeing memory)
if (size < 0) { if (memstoreDataSize < 0) {
LOG.error("Asked to modify this region's (" + this.toString() LOG.error("Asked to modify this region's (" + this.toString()
+ ") memstoreSize to a negative value which is incorrect. Current memstoreSize=" + ") memstoreSize to a negative value which is incorrect. Current memstoreSize="
+ (size-memStoreSize) + ", delta=" + memStoreSize, new Exception()); + (memstoreDataSize - delta) + ", delta=" + delta, new Exception());
} }
return size;
} }
@Override @Override
@ -1180,7 +1192,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override @Override
public long getMemstoreSize() { public long getMemstoreSize() {
return memstoreSize.get(); return memstoreDataSize.get();
} }
@Override @Override
@ -1490,7 +1502,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
int failedfFlushCount = 0; int failedfFlushCount = 0;
int flushCount = 0; int flushCount = 0;
long tmp = 0; long tmp = 0;
long remainingSize = this.memstoreSize.get(); long remainingSize = this.memstoreDataSize.get();
while (remainingSize > 0) { while (remainingSize > 0) {
try { try {
internalFlushcache(status); internalFlushcache(status);
@ -1499,7 +1511,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
" (carrying snapshot?) " + this); " (carrying snapshot?) " + this);
} }
flushCount++; flushCount++;
tmp = this.memstoreSize.get(); tmp = this.memstoreDataSize.get();
if (tmp >= remainingSize) { if (tmp >= remainingSize) {
failedfFlushCount++; failedfFlushCount++;
} }
@ -1534,8 +1546,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// close each store in parallel // close each store in parallel
for (final Store store : stores.values()) { for (final Store store : stores.values()) {
long flushableSize = store.getFlushableSize(); MemstoreSize flushableSize = store.getSizeToFlush();
if (!(abort || flushableSize == 0 || writestate.readOnly)) { if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) {
if (getRegionServerServices() != null) { if (getRegionServerServices() != null) {
getRegionServerServices().abort("Assertion failed while closing store " getRegionServerServices().abort("Assertion failed while closing store "
+ getRegionInfo().getRegionNameAsString() + " " + store + getRegionInfo().getRegionNameAsString() + " " + store
@ -1580,9 +1592,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.closed.set(true); this.closed.set(true);
if (!canFlush) { if (!canFlush) {
addAndGetGlobalMemstoreSize(-memstoreSize.get()); this.decrMemstoreSize(new MemstoreSize(memstoreDataSize.get(), getMemstoreHeapOverhead()));
} else if (memstoreSize.get() != 0) { } else if (memstoreDataSize.get() != 0) {
LOG.error("Memstore size is " + memstoreSize.get()); LOG.error("Memstore size is " + memstoreDataSize.get());
} }
if (coprocessorHost != null) { if (coprocessorHost != null) {
status.setStatus("Running coprocessor post-close hooks"); status.setStatus("Running coprocessor post-close hooks");
@ -1605,6 +1617,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
} }
private long getMemstoreHeapOverhead() {
long overhead = 0;
for (Store s : this.stores.values()) {
overhead += s.getSizeOfMemStore().getHeapOverhead();
}
return overhead;
}
@Override @Override
public void waitForFlushesAndCompactions() { public void waitForFlushesAndCompactions() {
synchronized (writestate) { synchronized (writestate) {
@ -1670,7 +1690,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return True if its worth doing a flush before we put up the close flag. * @return True if its worth doing a flush before we put up the close flag.
*/ */
private boolean worthPreFlushing() { private boolean worthPreFlushing() {
return this.memstoreSize.get() > return this.memstoreDataSize.get() >
this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5); this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
} }
@ -2246,12 +2266,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
// to no other that it can use to associate with the bulk load. Hence this little dance below // to no other that it can use to associate with the bulk load. Hence this little dance below
// to go get one. // to go get one.
if (this.memstoreSize.get() <= 0) { if (this.memstoreDataSize.get() <= 0) {
// Take an update lock so no edits can come into memory just yet. // Take an update lock so no edits can come into memory just yet.
this.updatesLock.writeLock().lock(); this.updatesLock.writeLock().lock();
WriteEntry writeEntry = null; WriteEntry writeEntry = null;
try { try {
if (this.memstoreSize.get() <= 0) { if (this.memstoreDataSize.get() <= 0) {
// Presume that if there are still no edits in the memstore, then there are no edits for // Presume that if there are still no edits in the memstore, then there are no edits for
// this region out in the WAL subsystem so no need to do any trickery clearing out // this region out in the WAL subsystem so no need to do any trickery clearing out
// edits in the WAL sub-system. Up the sequence number so the resulting flush id is for // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
@ -2294,7 +2314,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// block waiting for the lock for internal flush // block waiting for the lock for internal flush
this.updatesLock.writeLock().lock(); this.updatesLock.writeLock().lock();
status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName()); status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
long totalFlushableSizeOfFlushableStores = 0; MemstoreSize totalSizeOfFlushableStores = new MemstoreSize();
Set<byte[]> flushedFamilyNames = new HashSet<byte[]>(); Set<byte[]> flushedFamilyNames = new HashSet<byte[]>();
for (Store store: storesToFlush) { for (Store store: storesToFlush) {
@ -2305,8 +2325,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
= new TreeMap<byte[], StoreFlushContext>(Bytes.BYTES_COMPARATOR); = new TreeMap<byte[], StoreFlushContext>(Bytes.BYTES_COMPARATOR);
TreeMap<byte[], List<Path>> committedFiles = new TreeMap<byte[], List<Path>>( TreeMap<byte[], List<Path>> committedFiles = new TreeMap<byte[], List<Path>>(
Bytes.BYTES_COMPARATOR); Bytes.BYTES_COMPARATOR);
TreeMap<byte[], Long> storeFlushableSize TreeMap<byte[], MemstoreSize> storeFlushableSize
= new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR); = new TreeMap<byte[], MemstoreSize>(Bytes.BYTES_COMPARATOR);
// The sequence id of this flush operation which is used to log FlushMarker and pass to // The sequence id of this flush operation which is used to log FlushMarker and pass to
// createFlushContext to use as the store file's sequence id. It can be in advance of edits // createFlushContext to use as the store file's sequence id. It can be in advance of edits
// still in the memstore, edits that are in other column families yet to be flushed. // still in the memstore, edits that are in other column families yet to be flushed.
@ -2338,10 +2358,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
for (Store s : storesToFlush) { for (Store s : storesToFlush) {
totalFlushableSizeOfFlushableStores += s.getFlushableSize(); MemstoreSize flushableSize = s.getSizeToFlush();
totalSizeOfFlushableStores.incMemstoreSize(flushableSize);
storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId)); storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize()); storeFlushableSize.put(s.getFamily().getName(), flushableSize);
} }
// write the snapshot start to WAL // write the snapshot start to WAL
@ -2364,11 +2385,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.updatesLock.writeLock().unlock(); this.updatesLock.writeLock().unlock();
} }
String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " + String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " +
"flushsize=" + totalFlushableSizeOfFlushableStores; "flushsize=" + totalSizeOfFlushableStores;
status.setStatus(s); status.setStatus(s);
doSyncOfUnflushedWALChanges(wal, getRegionInfo()); doSyncOfUnflushedWALChanges(wal, getRegionInfo());
return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
flushOpSeqId, flushedSeqId, totalFlushableSizeOfFlushableStores); flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores);
} }
/** /**
@ -2384,11 +2405,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
perCfExtras = new StringBuilder(); perCfExtras = new StringBuilder();
for (Store store: storesToFlush) { for (Store store: storesToFlush) {
perCfExtras.append("; ").append(store.getColumnFamilyName()); perCfExtras.append("; ").append(store.getColumnFamilyName());
perCfExtras.append("=").append(StringUtils.byteDesc(store.getFlushableSize())); perCfExtras.append("=")
.append(StringUtils.byteDesc(store.getSizeToFlush().getDataSize()));
} }
} }
LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() + LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
" column families, memstore=" + StringUtils.byteDesc(this.memstoreSize.get()) + " column families, memstore=" + StringUtils.byteDesc(this.memstoreDataSize.get()) +
((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") + ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId)); ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
} }
@ -2468,7 +2490,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
long startTime = prepareResult.startTime; long startTime = prepareResult.startTime;
long flushOpSeqId = prepareResult.flushOpSeqId; long flushOpSeqId = prepareResult.flushOpSeqId;
long flushedSeqId = prepareResult.flushedSeqId; long flushedSeqId = prepareResult.flushedSeqId;
long totalFlushableSizeOfFlushableStores = prepareResult.totalFlushableSize;
String s = "Flushing stores of " + this; String s = "Flushing stores of " + this;
status.setStatus(s); status.setStatus(s);
@ -2504,14 +2525,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
committedFiles.put(storeName, storeCommittedFiles); committedFiles.put(storeName, storeCommittedFiles);
// Flush committed no files, indicating flush is empty or flush was canceled // Flush committed no files, indicating flush is empty or flush was canceled
if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) { if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
totalFlushableSizeOfFlushableStores -= prepareResult.storeFlushableSize.get(storeName); MemstoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName);
prepareResult.totalFlushableSize.decMemstoreSize(storeFlushableSize);
} }
flushedOutputFileSize += flush.getOutputFileSize(); flushedOutputFileSize += flush.getOutputFileSize();
} }
storeFlushCtxs.clear(); storeFlushCtxs.clear();
// Set down the memstore size by amount of flush. // Set down the memstore size by amount of flush.
this.addAndGetGlobalMemstoreSize(-totalFlushableSizeOfFlushableStores); this.decrMemstoreSize(prepareResult.totalFlushableSize);
if (wal != null) { if (wal != null) {
// write flush marker to WAL. If fail, we should throw DroppedSnapshotException // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
@ -2581,10 +2603,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
long time = EnvironmentEdgeManager.currentTime() - startTime; long time = EnvironmentEdgeManager.currentTime() - startTime;
long memstoresize = this.memstoreSize.get(); long memstoresize = this.memstoreDataSize.get();
String msg = "Finished memstore flush of ~" String msg = "Finished memstore flush of ~"
+ StringUtils.byteDesc(totalFlushableSizeOfFlushableStores) + "/" + StringUtils.byteDesc(prepareResult.totalFlushableSize.getDataSize()) + "/"
+ totalFlushableSizeOfFlushableStores + ", currentsize=" + prepareResult.totalFlushableSize.getDataSize() + ", currentsize="
+ StringUtils.byteDesc(memstoresize) + "/" + memstoresize + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
+ " for region " + this + " in " + time + "ms, sequenceid=" + " for region " + this + " in " + time + "ms, sequenceid="
+ flushOpSeqId + ", compaction requested=" + compactionRequested + flushOpSeqId + ", compaction requested=" + compactionRequested
@ -2594,7 +2616,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (rsServices != null && rsServices.getMetrics() != null) { if (rsServices != null && rsServices.getMetrics() != null) {
rsServices.getMetrics().updateFlush(time - startTime, rsServices.getMetrics().updateFlush(time - startTime,
totalFlushableSizeOfFlushableStores, flushedOutputFileSize); prepareResult.totalFlushableSize.getDataSize(), flushedOutputFileSize);
} }
return new FlushResultImpl(compactionRequested ? return new FlushResultImpl(compactionRequested ?
@ -3029,7 +3051,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
justification="Findbugs seems to be confused on this.") justification="Findbugs seems to be confused on this.")
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
// TODO: This needs a rewrite. Doesn't have to be this long. St.Ack 20160120 // TODO: This needs a rewrite. Doesn't have to be this long. St.Ack 20160120
private long doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException { private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
boolean replay = batchOp.isInReplay(); boolean replay = batchOp.isInReplay();
// Variable to note if all Put items are for the same CF -- metrics related // Variable to note if all Put items are for the same CF -- metrics related
boolean putsCfSetConsistent = true; boolean putsCfSetConsistent = true;
@ -3055,7 +3077,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
int cellCount = 0; int cellCount = 0;
/** Keep track of the locks we hold so we can release them in finally clause */ /** Keep track of the locks we hold so we can release them in finally clause */
List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length); List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
long addedSize = 0; MemstoreSize memstoreSize = new MemstoreSize();
try { try {
// STEP 1. Try to acquire as many locks as we can, and ensure we acquire at least one. // STEP 1. Try to acquire as many locks as we can, and ensure we acquire at least one.
int numReadyToWrite = 0; int numReadyToWrite = 0;
@ -3117,7 +3139,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily? // Nothing to put/delete -- an exception in the above such as NoSuchColumnFamily?
if (numReadyToWrite <= 0) { if (numReadyToWrite <= 0) {
return 0L; return;
} }
for (int i = firstIndex; !replay && i < lastIndexExclusive; i++) { for (int i = firstIndex; !replay && i < lastIndexExclusive; i++) {
@ -3155,7 +3177,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(), new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
if (coprocessorHost.preBatchMutate(miniBatchOp)) { if (coprocessorHost.preBatchMutate(miniBatchOp)) {
return 0L; return;
} else { } else {
for (int i = firstIndex; i < lastIndexExclusive; i++) { for (int i = firstIndex; i < lastIndexExclusive; i++) {
if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) { if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
@ -3303,7 +3325,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.updateSequenceId(familyMaps[i].values(), this.updateSequenceId(familyMaps[i].values(),
replay? batchOp.getReplaySequenceId(): writeEntry.getWriteNumber()); replay? batchOp.getReplaySequenceId(): writeEntry.getWriteNumber());
} }
addedSize += applyFamilyMapToMemstore(familyMaps[i]); applyFamilyMapToMemstore(familyMaps[i], memstoreSize);
} }
// STEP 6. Complete mvcc. // STEP 6. Complete mvcc.
@ -3355,11 +3377,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
success = true; success = true;
return addedSize;
} finally { } finally {
// Call complete rather than completeAndWait because we probably had error if walKey != null // Call complete rather than completeAndWait because we probably had error if walKey != null
if (writeEntry != null) mvcc.complete(writeEntry); if (writeEntry != null) mvcc.complete(writeEntry);
this.addAndGetGlobalMemstoreSize(addedSize); this.addAndGetMemstoreSize(memstoreSize);
if (locked) { if (locked) {
this.updatesLock.readLock().unlock(); this.updatesLock.readLock().unlock();
} }
@ -3778,7 +3799,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// If catalog region, do not impose resource constraints or block updates. // If catalog region, do not impose resource constraints or block updates.
if (this.getRegionInfo().isMetaRegion()) return; if (this.getRegionInfo().isMetaRegion()) return;
if (this.memstoreSize.get() > this.blockingMemStoreSize) { if (this.memstoreDataSize.get() > this.blockingMemStoreSize) {
blockedRequestsCount.increment(); blockedRequestsCount.increment();
requestFlush(); requestFlush();
throw new RegionTooBusyException("Above memstore limit, " + throw new RegionTooBusyException("Above memstore limit, " +
@ -3786,7 +3807,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.getRegionInfo().getRegionNameAsString()) + this.getRegionInfo().getRegionNameAsString()) +
", server=" + (this.getRegionServerServices() == null ? "unknown" : ", server=" + (this.getRegionServerServices() == null ? "unknown" :
this.getRegionServerServices().getServerName()) + this.getRegionServerServices().getServerName()) +
", memstoreSize=" + memstoreSize.get() + ", memstoreSize=" + memstoreDataSize.get() +
", blockingMemStoreSize=" + blockingMemStoreSize); ", blockingMemStoreSize=" + blockingMemStoreSize);
} }
} }
@ -3831,57 +3852,53 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
doBatchMutate(p); doBatchMutate(p);
} }
/** /*
* Atomically apply the given map of family->edits to the memstore. * Atomically apply the given map of family->edits to the memstore.
* This handles the consistency control on its own, but the caller * This handles the consistency control on its own, but the caller
* should already have locked updatesLock.readLock(). This also does * should already have locked updatesLock.readLock(). This also does
* <b>not</b> check the families for validity. * <b>not</b> check the families for validity.
* *
* @param familyMap Map of Cells by family * @param familyMap Map of Cells by family
* @return the additional memory usage of the memstore caused by the new entries. * @param memstoreSize
*/ */
private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap) private void applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap,
throws IOException { MemstoreSize memstoreSize) throws IOException {
long size = 0;
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) { for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
byte[] family = e.getKey(); byte[] family = e.getKey();
List<Cell> cells = e.getValue(); List<Cell> cells = e.getValue();
assert cells instanceof RandomAccess; assert cells instanceof RandomAccess;
size += applyToMemstore(getStore(family), cells, false); applyToMemstore(getStore(family), cells, false, memstoreSize);
} }
return size;
} }
/** /*
* @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
* set; when set we will run operations that make sense in the increment/append scenario but * set; when set we will run operations that make sense in the increment/append scenario but
* that do not make sense otherwise. * that do not make sense otherwise.
* @return Memstore change in size on insert of these Cells.
* @see #applyToMemstore(Store, Cell, long) * @see #applyToMemstore(Store, Cell, long)
*/ */
private long applyToMemstore(final Store store, final List<Cell> cells, final boolean delta) private void applyToMemstore(final Store store, final List<Cell> cells, final boolean delta,
throws IOException { MemstoreSize memstoreSize) throws IOException {
// Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!! // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
boolean upsert = delta && store.getFamily().getMaxVersions() == 1; boolean upsert = delta && store.getFamily().getMaxVersions() == 1;
if (upsert) { if (upsert) {
return ((HStore) store).upsert(cells, getSmallestReadPoint()); ((HStore) store).upsert(cells, getSmallestReadPoint(), memstoreSize);
} else { } else {
return ((HStore) store).add(cells); ((HStore) store).add(cells, memstoreSize);
} }
} }
/** /*
* @return Memstore change in size on insert of these Cells.
* @see #applyToMemstore(Store, List, boolean, boolean, long) * @see #applyToMemstore(Store, List, boolean, boolean, long)
*/ */
private long applyToMemstore(final Store store, final Cell cell) private void applyToMemstore(final Store store, final Cell cell, MemstoreSize memstoreSize)
throws IOException { throws IOException {
// Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!! // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!!
if (store == null) { if (store == null) {
checkFamily(CellUtil.cloneFamily(cell)); checkFamily(CellUtil.cloneFamily(cell));
// Unreachable because checkFamily will throw exception // Unreachable because checkFamily will throw exception
} }
return ((HStore) store).add(cell); ((HStore) store).add(cell, memstoreSize);
} }
@Override @Override
@ -4200,6 +4217,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
boolean flush = false; boolean flush = false;
MemstoreSize memstoreSize = new MemstoreSize();
for (Cell cell: val.getCells()) { for (Cell cell: val.getCells()) {
// Check this edit is for me. Also, guard against writing the special // Check this edit is for me. Also, guard against writing the special
// METACOLUMN info such as HBASE::CACHEFLUSH entries // METACOLUMN info such as HBASE::CACHEFLUSH entries
@ -4241,12 +4259,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
CellUtil.setSequenceId(cell, currentReplaySeqId); CellUtil.setSequenceId(cell, currentReplaySeqId);
// Once we are over the limit, restoreEdit will keep returning true to restoreEdit(store, cell, memstoreSize);
// flush -- but don't flush until we've played all the kvs that make up
// the WALEdit.
flush |= restoreEdit(store, cell);
editsCount++; editsCount++;
} }
if (this.rsAccounting != null) {
rsAccounting.addRegionReplayEditsSize(getRegionInfo().getRegionName(),
memstoreSize);
}
flush = isFlushSize(this.addAndGetMemstoreSize(memstoreSize));
if (flush) { if (flush) {
internalFlushcache(null, currentEditSeqId, stores.values(), status, false); internalFlushcache(null, currentEditSeqId, stores.values(), status, false);
} }
@ -4555,7 +4575,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
replayFlushInStores(flush, prepareFlushResult, true); replayFlushInStores(flush, prepareFlushResult, true);
// Set down the memstore size by amount of flush. // Set down the memstore size by amount of flush.
this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize); this.decrMemstoreSize(prepareFlushResult.totalFlushableSize);
this.prepareFlushResult = null; this.prepareFlushResult = null;
writestate.flushing = false; writestate.flushing = false;
@ -4588,7 +4608,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
replayFlushInStores(flush, prepareFlushResult, true); replayFlushInStores(flush, prepareFlushResult, true);
// Set down the memstore size by amount of flush. // Set down the memstore size by amount of flush.
this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize); this.decrMemstoreSize(prepareFlushResult.totalFlushableSize);
// Inspect the memstore contents to see whether the memstore contains only edits // Inspect the memstore contents to see whether the memstore contains only edits
// with seqId smaller than the flush seqId. If so, we can discard those edits. // with seqId smaller than the flush seqId. If so, we can discard those edits.
@ -4691,8 +4711,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* if the memstore edits have seqNums smaller than the given seq id * if the memstore edits have seqNums smaller than the given seq id
* @throws IOException * @throws IOException
*/ */
private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException { private MemstoreSize dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
long totalFreedSize = 0; MemstoreSize totalFreedSize = new MemstoreSize();
this.updatesLock.writeLock().lock(); this.updatesLock.writeLock().lock();
try { try {
@ -4706,10 +4726,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Prepare flush (take a snapshot) and then abort (drop the snapshot) // Prepare flush (take a snapshot) and then abort (drop the snapshot)
if (store == null) { if (store == null) {
for (Store s : stores.values()) { for (Store s : stores.values()) {
totalFreedSize += doDropStoreMemstoreContentsForSeqId(s, currentSeqId); totalFreedSize.incMemstoreSize(doDropStoreMemstoreContentsForSeqId(s, currentSeqId));
} }
} else { } else {
totalFreedSize += doDropStoreMemstoreContentsForSeqId(store, currentSeqId); totalFreedSize.incMemstoreSize(doDropStoreMemstoreContentsForSeqId(store, currentSeqId));
} }
} else { } else {
LOG.info(getRegionInfo().getEncodedName() + " : " LOG.info(getRegionInfo().getEncodedName() + " : "
@ -4722,13 +4742,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return totalFreedSize; return totalFreedSize;
} }
private long doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId) throws IOException { private MemstoreSize doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId)
long snapshotSize = s.getFlushableSize(); throws IOException {
this.addAndGetGlobalMemstoreSize(-snapshotSize); MemstoreSize flushableSize = s.getSizeToFlush();
this.decrMemstoreSize(flushableSize);
StoreFlushContext ctx = s.createFlushContext(currentSeqId); StoreFlushContext ctx = s.createFlushContext(currentSeqId);
ctx.prepare(); ctx.prepare();
ctx.abort(); ctx.abort();
return snapshotSize; return flushableSize;
} }
private void replayWALFlushAbortMarker(FlushDescriptor flush) { private void replayWALFlushAbortMarker(FlushDescriptor flush) {
@ -4841,9 +4862,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
null : this.prepareFlushResult.storeFlushCtxs.get(family); null : this.prepareFlushResult.storeFlushCtxs.get(family);
if (ctx != null) { if (ctx != null) {
long snapshotSize = store.getFlushableSize(); MemstoreSize snapshotSize = store.getSizeToFlush();
ctx.abort(); ctx.abort();
this.addAndGetGlobalMemstoreSize(-snapshotSize); this.decrMemstoreSize(snapshotSize);
this.prepareFlushResult.storeFlushCtxs.remove(family); this.prepareFlushResult.storeFlushCtxs.remove(family);
} }
} }
@ -4972,7 +4993,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (store == null) { if (store == null) {
continue; continue;
} }
if (store.getSnapshotSize() > 0) { if (store.getSizeOfSnapshot().getDataSize() > 0) {
canDrop = false; canDrop = false;
break; break;
} }
@ -5005,7 +5026,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
+ "Refreshing store files to see whether we can free up memstore"); + "Refreshing store files to see whether we can free up memstore");
} }
long totalFreedSize = 0; long totalFreedDataSize = 0;
long smallestSeqIdInStores = Long.MAX_VALUE; long smallestSeqIdInStores = Long.MAX_VALUE;
@ -5035,11 +5056,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName()); null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
if (ctx != null) { if (ctx != null) {
long snapshotSize = store.getFlushableSize(); MemstoreSize snapshotSize = store.getSizeToFlush();
ctx.abort(); ctx.abort();
this.addAndGetGlobalMemstoreSize(-snapshotSize); this.decrMemstoreSize(snapshotSize);
this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName()); this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
totalFreedSize += snapshotSize; totalFreedDataSize += snapshotSize.getDataSize();
} }
} }
} }
@ -5071,7 +5092,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (!force) { if (!force) {
for (Map.Entry<Store, Long> entry : map.entrySet()) { for (Map.Entry<Store, Long> entry : map.entrySet()) {
// Drop the memstore contents if they are now smaller than the latest seen flushed file // Drop the memstore contents if they are now smaller than the latest seen flushed file
totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey()); totalFreedDataSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey())
.getDataSize();
} }
} else { } else {
synchronized (storeSeqIds) { synchronized (storeSeqIds) {
@ -5085,7 +5107,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
synchronized (this) { synchronized (this) {
notifyAll(); // FindBugs NN_NAKED_NOTIFY notifyAll(); // FindBugs NN_NAKED_NOTIFY
} }
return totalFreedSize > 0; return totalFreedDataSize > 0;
} finally { } finally {
closeRegionOperation(); closeRegionOperation();
} }
@ -5124,18 +5146,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
+ " does not match this region: " + this.getRegionInfo()); + " does not match this region: " + this.getRegionInfo());
} }
/** /*
* Used by tests * Used by tests
* @param s Store to add edit too. * @param s Store to add edit too.
* @param cell Cell to add. * @param cell Cell to add.
* @return True if we should flush. * @param memstoreSize
*/ */
protected boolean restoreEdit(final HStore s, final Cell cell) { protected void restoreEdit(final HStore s, final Cell cell, MemstoreSize memstoreSize) {
long kvSize = s.add(cell); s.add(cell, memstoreSize);
if (this.rsAccounting != null) {
rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize);
}
return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
} }
/* /*
@ -6986,7 +7004,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return null; return null;
} }
ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder(); ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreSize.get() * 100) / this stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreDataSize.get() * 100) / this
.memstoreFlushSize))); .memstoreFlushSize)));
stats.setHeapOccupancy((int)rsServices.getHeapMemoryManager().getHeapOccupancyPercent()*100); stats.setHeapOccupancy((int)rsServices.getHeapMemoryManager().getHeapOccupancyPercent()*100);
stats.setCompactionPressure((int)rsServices.getCompactionPressure()*100 > 100 ? 100 : stats.setCompactionPressure((int)rsServices.getCompactionPressure()*100 > 100 ? 100 :
@ -7035,12 +7053,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
boolean locked; boolean locked;
List<RowLock> acquiredRowLocks; List<RowLock> acquiredRowLocks;
long addedSize = 0;
List<Mutation> mutations = new ArrayList<Mutation>(); List<Mutation> mutations = new ArrayList<Mutation>();
Collection<byte[]> rowsToLock = processor.getRowsToLock(); Collection<byte[]> rowsToLock = processor.getRowsToLock();
// This is assigned by mvcc either explicity in the below or in the guts of the WAL append // This is assigned by mvcc either explicity in the below or in the guts of the WAL append
// when it assigns the edit a sequencedid (A.K.A the mvcc write number). // when it assigns the edit a sequencedid (A.K.A the mvcc write number).
WriteEntry writeEntry = null; WriteEntry writeEntry = null;
MemstoreSize memstoreSize = new MemstoreSize();
try { try {
// STEP 2. Acquire the row lock(s) // STEP 2. Acquire the row lock(s)
acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size()); acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
@ -7084,7 +7102,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// If no WAL, need to stamp it here. // If no WAL, need to stamp it here.
CellUtil.setSequenceId(cell, sequenceId); CellUtil.setSequenceId(cell, sequenceId);
} }
addedSize += applyToMemstore(getHStore(cell), cell); applyToMemstore(getHStore(cell), cell, memstoreSize);
} }
} }
// STEP 8. Complete mvcc. // STEP 8. Complete mvcc.
@ -7119,7 +7137,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} finally { } finally {
closeRegionOperation(); closeRegionOperation();
if (!mutations.isEmpty()) { if (!mutations.isEmpty()) {
long newSize = this.addAndGetGlobalMemstoreSize(addedSize); long newSize = this.addAndGetMemstoreSize(memstoreSize);
requestFlushIfNeeded(newSize); requestFlushIfNeeded(newSize);
} }
} }
@ -7203,7 +7221,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// See HBASE-16304 // See HBASE-16304
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private void dropMemstoreContents() throws IOException { private void dropMemstoreContents() throws IOException {
long totalFreedSize = 0; MemstoreSize totalFreedSize = new MemstoreSize();
while (!storeSeqIds.isEmpty()) { while (!storeSeqIds.isEmpty()) {
Map<Store, Long> map = null; Map<Store, Long> map = null;
synchronized (storeSeqIds) { synchronized (storeSeqIds) {
@ -7212,11 +7230,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
for (Map.Entry<Store, Long> entry : map.entrySet()) { for (Map.Entry<Store, Long> entry : map.entrySet()) {
// Drop the memstore contents if they are now smaller than the latest seen flushed file // Drop the memstore contents if they are now smaller than the latest seen flushed file
totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey()); totalFreedSize
.incMemstoreSize(dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey()));
} }
} }
if (totalFreedSize > 0) { if (totalFreedSize.getDataSize() > 0) {
LOG.debug("Freed " + totalFreedSize + " bytes from memstore"); LOG.debug("Freed " + totalFreedSize.getDataSize() + " bytes from memstore");
} }
} }
@ -7237,8 +7256,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* on the passed in <code>op</code> to do increment or append specific paths. * on the passed in <code>op</code> to do increment or append specific paths.
*/ */
private Result doDelta(Operation op, Mutation mutation, long nonceGroup, long nonce, private Result doDelta(Operation op, Mutation mutation, long nonceGroup, long nonce,
boolean returnResults) boolean returnResults) throws IOException {
throws IOException {
checkReadOnly(); checkReadOnly();
checkResources(); checkResources();
checkRow(mutation.getRow(), op.toString()); checkRow(mutation.getRow(), op.toString());
@ -7246,9 +7264,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
WriteEntry writeEntry = null; WriteEntry writeEntry = null;
startRegionOperation(op); startRegionOperation(op);
long accumulatedResultSize = 0;
List<Cell> results = returnResults? new ArrayList<Cell>(mutation.size()): null; List<Cell> results = returnResults? new ArrayList<Cell>(mutation.size()): null;
RowLock rowLock = getRowLockInternal(mutation.getRow(), false); RowLock rowLock = getRowLockInternal(mutation.getRow(), false);
MemstoreSize memstoreSize = new MemstoreSize();
try { try {
lock(this.updatesLock.readLock()); lock(this.updatesLock.readLock());
try { try {
@ -7273,8 +7291,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
updateSequenceId(forMemStore.values(), writeEntry.getWriteNumber()); updateSequenceId(forMemStore.values(), writeEntry.getWriteNumber());
} }
// Now write to MemStore. Do it a column family at a time. // Now write to MemStore. Do it a column family at a time.
for (Map.Entry<Store, List<Cell>> e: forMemStore.entrySet()) { for (Map.Entry<Store, List<Cell>> e : forMemStore.entrySet()) {
accumulatedResultSize += applyToMemstore(e.getKey(), e.getValue(), true); applyToMemstore(e.getKey(), e.getValue(), true, memstoreSize);
} }
mvcc.completeAndWait(writeEntry); mvcc.completeAndWait(writeEntry);
if (rsServices != null && rsServices.getNonceManager() != null) { if (rsServices != null && rsServices.getNonceManager() != null) {
@ -7299,7 +7317,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (writeEntry != null) mvcc.complete(writeEntry); if (writeEntry != null) mvcc.complete(writeEntry);
rowLock.release(); rowLock.release();
// Request a cache flush if over the limit. Do it outside update lock. // Request a cache flush if over the limit. Do it outside update lock.
if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush(); if (isFlushSize(addAndGetMemstoreSize(memstoreSize))) {
requestFlush();
}
closeRegionOperation(op); closeRegionOperation(op);
if (this.metricsRegion != null) { if (this.metricsRegion != null) {
switch (op) { switch (op) {
@ -8155,7 +8175,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
for (Store s : getStores()) { for (Store s : getStores()) {
buf.append(s.getFamily().getNameAsString()); buf.append(s.getFamily().getNameAsString());
buf.append(" size: "); buf.append(" size: ");
buf.append(s.getMemStoreSize()); buf.append(s.getSizeOfMemStore().getDataSize());
buf.append(" "); buf.append(" ");
} }
buf.append("end-of-stores"); buf.append("end-of-stores");

View File

@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -362,12 +361,26 @@ public class HStore implements Store {
} }
@Override @Override
@Deprecated
public long getFlushableSize() { public long getFlushableSize() {
MemstoreSize size = getSizeToFlush();
return size.getDataSize() + size.getHeapOverhead();
}
@Override
public MemstoreSize getSizeToFlush() {
return this.memstore.getFlushableSize(); return this.memstore.getFlushableSize();
} }
@Override @Override
@Deprecated
public long getSnapshotSize() { public long getSnapshotSize() {
MemstoreSize size = getSizeOfSnapshot();
return size.getDataSize() + size.getHeapOverhead();
}
@Override
public MemstoreSize getSizeOfSnapshot() {
return this.memstore.getSnapshotSize(); return this.memstore.getSnapshotSize();
} }
@ -636,12 +649,12 @@ public class HStore implements Store {
/** /**
* Adds a value to the memstore * Adds a value to the memstore
* @param cell * @param cell
* @return memstore size delta * @param memstoreSize
*/ */
public long add(final Cell cell) { public void add(final Cell cell, MemstoreSize memstoreSize) {
lock.readLock().lock(); lock.readLock().lock();
try { try {
return this.memstore.add(cell); this.memstore.add(cell, memstoreSize);
} finally { } finally {
lock.readLock().unlock(); lock.readLock().unlock();
} }
@ -650,12 +663,12 @@ public class HStore implements Store {
/** /**
* Adds the specified value to the memstore * Adds the specified value to the memstore
* @param cells * @param cells
* @return memstore size delta * @param memstoreSize
*/ */
public long add(final Iterable<Cell> cells) { public void add(final Iterable<Cell> cells, MemstoreSize memstoreSize) {
lock.readLock().lock(); lock.readLock().lock();
try { try {
return memstore.add(cells); memstore.add(cells, memstoreSize);
} finally { } finally {
lock.readLock().unlock(); lock.readLock().unlock();
} }
@ -666,21 +679,6 @@ public class HStore implements Store {
return memstore.timeOfOldestEdit(); return memstore.timeOfOldestEdit();
} }
/**
* Adds a value to the memstore
*
* @param kv
* @return memstore size delta
*/
protected long delete(final KeyValue kv) {
lock.readLock().lock();
try {
return this.memstore.delete(kv);
} finally {
lock.readLock().unlock();
}
}
/** /**
* @return All store files. * @return All store files.
*/ */
@ -2026,7 +2024,14 @@ public class HStore implements Store {
} }
@Override @Override
@Deprecated
public long getMemStoreSize() { public long getMemStoreSize() {
MemstoreSize size = getSizeOfMemStore();
return size.getDataSize() + size.getHeapOverhead();
}
@Override
public MemstoreSize getSizeOfMemStore() {
return this.memstore.size(); return this.memstore.size();
} }
@ -2068,37 +2073,6 @@ public class HStore implements Store {
return this.region.getSmallestReadPoint(); return this.region.getSmallestReadPoint();
} }
/**
* Updates the value for the given row/family/qualifier. This function will always be seen as
* atomic by other readers because it only puts a single KV to memstore. Thus no read/write
* control necessary.
* @param row row to update
* @param f family to update
* @param qualifier qualifier to update
* @param newValue the new value to set into memstore
* @return memstore size delta
* @throws IOException
*/
@VisibleForTesting
public long updateColumnValue(byte [] row, byte [] f,
byte [] qualifier, long newValue)
throws IOException {
this.lock.readLock().lock();
try {
long now = EnvironmentEdgeManager.currentTime();
return this.memstore.updateColumnValue(row,
f,
qualifier,
newValue,
now);
} finally {
this.lock.readLock().unlock();
}
}
/** /**
* Adds or replaces the specified KeyValues. * Adds or replaces the specified KeyValues.
* <p> * <p>
@ -2109,13 +2083,14 @@ public class HStore implements Store {
* across all of them. * across all of them.
* @param cells * @param cells
* @param readpoint readpoint below which we can safely remove duplicate KVs * @param readpoint readpoint below which we can safely remove duplicate KVs
* @return memstore size delta * @param memstoreSize
* @throws IOException * @throws IOException
*/ */
public long upsert(Iterable<Cell> cells, long readpoint) throws IOException { public void upsert(Iterable<Cell> cells, long readpoint, MemstoreSize memstoreSize)
throws IOException {
this.lock.readLock().lock(); this.lock.readLock().lock();
try { try {
return this.memstore.upsert(cells, readpoint); this.memstore.upsert(cells, readpoint, memstoreSize);
} finally { } finally {
this.lock.readLock().unlock(); this.lock.readLock().unlock();
} }
@ -2149,7 +2124,7 @@ public class HStore implements Store {
// passing the current sequence number of the wal - to allow bookkeeping in the memstore // passing the current sequence number of the wal - to allow bookkeeping in the memstore
this.snapshot = memstore.snapshot(); this.snapshot = memstore.snapshot();
this.cacheFlushCount = snapshot.getCellsCount(); this.cacheFlushCount = snapshot.getCellsCount();
this.cacheFlushSize = snapshot.getSize(); this.cacheFlushSize = snapshot.getDataSize();
committedFiles = new ArrayList<Path>(1); committedFiles = new ArrayList<Path>(1);
} }
@ -2282,7 +2257,8 @@ public class HStore implements Store {
@Override @Override
public long heapSize() { public long heapSize() {
return DEEP_OVERHEAD + this.memstore.heapSize(); MemstoreSize memstoreSize = this.memstore.size();
return DEEP_OVERHEAD + memstoreSize.getDataSize() + memstoreSize.getHeapOverhead();
} }
@Override @Override

View File

@ -298,8 +298,10 @@ public class HeapMemoryManager {
metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt); metricsHeapMemoryManager.updateUnblockedFlushCount(unblockedFlushCnt);
tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize); tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize);
metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize()); metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize());
tunerContext.setCurMemStoreUsed((float)regionServerAccounting.getGlobalMemstoreSize() / maxHeapSize); long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemstoreSize()
metricsHeapMemoryManager.setCurMemStoreSizeGauge(regionServerAccounting.getGlobalMemstoreSize()); + regionServerAccounting.getGlobalMemstoreHeapOverhead();
tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize);
metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreHeapSize);
tunerContext.setCurBlockCacheSize(blockCachePercent); tunerContext.setCurBlockCacheSize(blockCachePercent);
tunerContext.setCurMemStoreSize(globalMemStorePercent); tunerContext.setCurMemStoreSize(globalMemStorePercent);
TunerResult result = null; TunerResult result = null;

View File

@ -21,7 +21,8 @@ package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -108,13 +109,14 @@ public class ImmutableSegment extends Segment {
super(new CellSet(comparator), // initiailize the CellSet with empty CellSet super(new CellSet(comparator), // initiailize the CellSet with empty CellSet
comparator, memStoreLAB); comparator, memStoreLAB);
type = Type.SKIPLIST_MAP_BASED; type = Type.SKIPLIST_MAP_BASED;
MemstoreSize memstoreSize = new MemstoreSize();
while (iterator.hasNext()) { while (iterator.hasNext()) {
Cell c = iterator.next(); Cell c = iterator.next();
// The scanner is doing all the elimination logic // The scanner is doing all the elimination logic
// now we just copy it to the new segment // now we just copy it to the new segment
Cell newKV = maybeCloneWithAllocator(c); Cell newKV = maybeCloneWithAllocator(c);
boolean usedMSLAB = (newKV != c); boolean usedMSLAB = (newKV != c);
internalAdd(newKV, usedMSLAB); // internalAdd(newKV, usedMSLAB, memstoreSize);
} }
this.timeRange = this.timeRangeTracker == null ? null : this.timeRangeTracker.toTimeRange(); this.timeRange = this.timeRangeTracker == null ? null : this.timeRangeTracker.toTimeRange();
} }
@ -140,19 +142,6 @@ public class ImmutableSegment extends Segment {
return this.timeRange.getMin(); return this.timeRange.getMin();
} }
@Override
public long size() {
switch (this.type) {
case SKIPLIST_MAP_BASED:
return keySize() + DEEP_OVERHEAD_CSLM;
case ARRAY_MAP_BASED:
return keySize() + DEEP_OVERHEAD_CAM;
default:
throw new RuntimeException("Unknown type " + type);
}
}
/**------------------------------------------------------------------------ /**------------------------------------------------------------------------
* Change the CellSet of this ImmutableSegment from one based on ConcurrentSkipListMap to one * Change the CellSet of this ImmutableSegment from one based on ConcurrentSkipListMap to one
* based on CellArrayMap. * based on CellArrayMap.
@ -164,7 +153,7 @@ public class ImmutableSegment extends Segment {
* thread of compaction, but to be on the safe side the initial CellSet is locally saved * thread of compaction, but to be on the safe side the initial CellSet is locally saved
* before the flattening and then replaced using CAS instruction. * before the flattening and then replaced using CAS instruction.
*/ */
public boolean flatten() { public boolean flatten(MemstoreSize memstoreSize) {
if (isFlat()) return false; if (isFlat()) return false;
CellSet oldCellSet = getCellSet(); CellSet oldCellSet = getCellSet();
int numOfCells = getCellsCount(); int numOfCells = getCellsCount();
@ -176,12 +165,13 @@ public class ImmutableSegment extends Segment {
// arrange the meta-data size, decrease all meta-data sizes related to SkipList // arrange the meta-data size, decrease all meta-data sizes related to SkipList
// (recreateCellArrayMapSet doesn't take the care for the sizes) // (recreateCellArrayMapSet doesn't take the care for the sizes)
long newSegmentSizeDelta = -(ClassSize.CONCURRENT_SKIPLISTMAP + long newSegmentSizeDelta = -(numOfCells * ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
numOfCells * ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
// add size of CellArrayMap and meta-data overhead per Cell // add size of CellArrayMap and meta-data overhead per Cell
newSegmentSizeDelta = newSegmentSizeDelta + ClassSize.CELL_ARRAY_MAP + newSegmentSizeDelta = newSegmentSizeDelta + numOfCells * ClassSize.CELL_ARRAY_MAP_ENTRY;
numOfCells * ClassSize.CELL_ARRAY_MAP_ENTRY; incSize(0, newSegmentSizeDelta);
incSize(newSegmentSizeDelta); if (memstoreSize != null) {
memstoreSize.incMemstoreSize(0, newSegmentSizeDelta);
}
return true; return true;
} }
@ -208,7 +198,7 @@ public class ImmutableSegment extends Segment {
boolean useMSLAB = (getMemStoreLAB()!=null); boolean useMSLAB = (getMemStoreLAB()!=null);
// second parameter true, because in compaction addition of the cell to new segment // second parameter true, because in compaction addition of the cell to new segment
// is always successful // is always successful
updateMetaInfo(c, true, useMSLAB); // updates the size per cell updateMetaInfo(c, true, useMSLAB, null); // updates the size per cell
i++; i++;
} }
// build the immutable CellSet // build the immutable CellSet
@ -216,14 +206,18 @@ public class ImmutableSegment extends Segment {
return new CellSet(cam); return new CellSet(cam);
} }
protected long heapSizeChange(Cell cell, boolean succ) { @Override
protected long heapOverheadChange(Cell cell, boolean succ) {
if (succ) { if (succ) {
switch (this.type) { switch (this.type) {
case SKIPLIST_MAP_BASED: case SKIPLIST_MAP_BASED:
return ClassSize return super.heapOverheadChange(cell, succ);
.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + CellUtil.estimatedHeapSizeOf(cell));
case ARRAY_MAP_BASED: case ARRAY_MAP_BASED:
return ClassSize.align(ClassSize.CELL_ARRAY_MAP_ENTRY + CellUtil.estimatedHeapSizeOf(cell)); if (cell instanceof ExtendedCell) {
return ClassSize
.align(ClassSize.CELL_ARRAY_MAP_ENTRY + ((ExtendedCell) cell).heapOverhead());
}
return ClassSize.align(ClassSize.CELL_ARRAY_MAP_ENTRY + KeyValue.FIXED_OVERHEAD);
} }
} }
return 0; return 0;

View File

@ -23,7 +23,6 @@ import java.util.List;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.io.HeapSize;
/** /**
* The MemStore holds in-memory modifications to the Store. Modifications are {@link Cell}s. * The MemStore holds in-memory modifications to the Store. Modifications are {@link Cell}s.
@ -33,7 +32,7 @@ import org.apache.hadoop.hbase.io.HeapSize;
* </p> * </p>
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public interface MemStore extends HeapSize { public interface MemStore {
/** /**
* Creates a snapshot of the current memstore. Snapshot must be cleared by call to * Creates a snapshot of the current memstore. Snapshot must be cleared by call to
@ -58,57 +57,35 @@ public interface MemStore extends HeapSize {
* *
* @return size of data that is going to be flushed * @return size of data that is going to be flushed
*/ */
long getFlushableSize(); MemstoreSize getFlushableSize();
/** /**
* Return the size of the snapshot(s) if any * Return the size of the snapshot(s) if any
* @return size of the memstore snapshot * @return size of the memstore snapshot
*/ */
long getSnapshotSize(); MemstoreSize getSnapshotSize();
/** /**
* Write an update * Write an update
* @param cell * @param cell
* @return approximate size of the passed cell. * @param memstoreSize The delta in memstore size will be passed back via this.
* This will include both data size and heap overhead delta.
*/ */
long add(final Cell cell); void add(final Cell cell, MemstoreSize memstoreSize);
/** /**
* Write the updates * Write the updates
* @param cells * @param cells
* @return approximate size of the passed cell. * @param memstoreSize The delta in memstore size will be passed back via this.
* This will include both data size and heap overhead delta.
*/ */
long add(Iterable<Cell> cells); void add(Iterable<Cell> cells, MemstoreSize memstoreSize);
/** /**
* @return Oldest timestamp of all the Cells in the MemStore * @return Oldest timestamp of all the Cells in the MemStore
*/ */
long timeOfOldestEdit(); long timeOfOldestEdit();
/**
* Write a delete
* @param deleteCell
* @return approximate size of the passed key and value.
*/
long delete(final Cell deleteCell);
/**
* Given the specs of a column, update it, first by inserting a new record,
* then removing the old one. Since there is only 1 KeyValue involved, the memstoreTS
* will be set to 0, thus ensuring that they instantly appear to anyone. The underlying
* store will ensure that the insert/delete each are atomic. A scanner/reader will either
* get the new value, or the old value and all readers will eventually only see the new
* value after the old was removed.
*
* @param row
* @param family
* @param qualifier
* @param newValue
* @param now
* @return Timestamp
*/
long updateColumnValue(byte[] row, byte[] family, byte[] qualifier, long newValue, long now);
/** /**
* Update or insert the specified cells. * Update or insert the specified cells.
* <p> * <p>
@ -122,9 +99,10 @@ public interface MemStore extends HeapSize {
* only see each KeyValue update as atomic. * only see each KeyValue update as atomic.
* @param cells * @param cells
* @param readpoint readpoint below which we can safely remove duplicate Cells. * @param readpoint readpoint below which we can safely remove duplicate Cells.
* @return change in memstore size * @param memstoreSize The delta in memstore size will be passed back via this.
* This will include both data size and heap overhead delta.
*/ */
long upsert(Iterable<Cell> cells, long readpoint); void upsert(Iterable<Cell> cells, long readpoint, MemstoreSize memstoreSize);
/** /**
* @return scanner over the memstore. This might include scanner over the snapshot when one is * @return scanner over the memstore. This might include scanner over the snapshot when one is
@ -133,13 +111,12 @@ public interface MemStore extends HeapSize {
List<KeyValueScanner> getScanners(long readPt) throws IOException; List<KeyValueScanner> getScanners(long readPt) throws IOException;
/** /**
* @return Total memory occupied by this MemStore. This includes active segment size and heap size * @return Total memory occupied by this MemStore. This won't include any size occupied by the
* overhead of this memstore but won't include any size occupied by the snapshot. We * snapshot. We assume the snapshot will get cleared soon. This is not thread safe and
* assume the snapshot will get cleared soon. This is not thread safe and the memstore may * the memstore may be changed while computing its size. It is the responsibility of the
* be changed while computing its size. It is the responsibility of the caller to make * caller to make sure this doesn't happen.
* sure this doesn't happen.
*/ */
long size(); MemstoreSize size();
/** /**
* This method is called when it is clear that the flush to disk is completed. * This method is called when it is clear that the flush to disk is completed.

View File

@ -197,9 +197,12 @@ class MemStoreFlusher implements FlushRequester {
ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) && ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) &&
(bestRegionReplica.getMemstoreSize() (bestRegionReplica.getMemstoreSize()
> secondaryMultiplier * regionToFlush.getMemstoreSize()))) { > secondaryMultiplier * regionToFlush.getMemstoreSize()))) {
LOG.info("Refreshing storefiles of region " + bestRegionReplica + LOG.info("Refreshing storefiles of region " + bestRegionReplica
" due to global heap pressure. memstore size=" + StringUtils.humanReadableInt( + " due to global heap pressure. Total memstore size="
server.getRegionServerAccounting().getGlobalMemstoreSize())); + StringUtils
.humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreSize())
+ " memstore heap overhead=" + StringUtils.humanReadableInt(
server.getRegionServerAccounting().getGlobalMemstoreHeapOverhead()));
flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica); flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica);
if (!flushedOne) { if (!flushedOne) {
LOG.info("Excluding secondary region " + bestRegionReplica + LOG.info("Excluding secondary region " + bestRegionReplica +
@ -343,16 +346,16 @@ class MemStoreFlusher implements FlushRequester {
* Return true if global memory usage is above the high watermark * Return true if global memory usage is above the high watermark
*/ */
private boolean isAboveHighWaterMark() { private boolean isAboveHighWaterMark() {
return server.getRegionServerAccounting(). return server.getRegionServerAccounting().getGlobalMemstoreSize()
getGlobalMemstoreSize() >= globalMemStoreLimit; + server.getRegionServerAccounting().getGlobalMemstoreHeapOverhead() >= globalMemStoreLimit;
} }
/** /**
* Return true if we're above the high watermark * Return true if we're above the high watermark
*/ */
private boolean isAboveLowWaterMark() { private boolean isAboveLowWaterMark() {
return server.getRegionServerAccounting(). return server.getRegionServerAccounting().getGlobalMemstoreSize() + server
getGlobalMemstoreSize() >= globalMemStoreLimitLowMark; .getRegionServerAccounting().getGlobalMemstoreHeapOverhead() >= globalMemStoreLimitLowMark;
} }
@Override @Override
@ -586,11 +589,13 @@ class MemStoreFlusher implements FlushRequester {
while (isAboveHighWaterMark() && !server.isStopped()) { while (isAboveHighWaterMark() && !server.isStopped()) {
if (!blocked) { if (!blocked) {
startTime = EnvironmentEdgeManager.currentTime(); startTime = EnvironmentEdgeManager.currentTime();
LOG.info("Blocking updates on " LOG.info("Blocking updates on " + server.toString() + ": the global memstore size "
+ server.toString() + TraditionalBinaryPrefix.long2String(
+ ": the global memstore size " server.getRegionServerAccounting().getGlobalMemstoreSize(), "", 1)
+ TraditionalBinaryPrefix.long2String(server.getRegionServerAccounting() + " + global memstore heap overhead "
.getGlobalMemstoreSize(), "", 1) + " is >= than blocking " + TraditionalBinaryPrefix.long2String(
server.getRegionServerAccounting().getGlobalMemstoreHeapOverhead(), "", 1)
+ " is >= than blocking "
+ TraditionalBinaryPrefix.long2String(globalMemStoreLimit, "", 1) + " size"); + TraditionalBinaryPrefix.long2String(globalMemStoreLimit, "", 1) + " size");
} }
blocked = true; blocked = true;

View File

@ -28,7 +28,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
public class MemStoreSnapshot { public class MemStoreSnapshot {
private final long id; private final long id;
private final int cellsCount; private final int cellsCount;
private final long size; private final long dataSize;
private final long heapOverhead;
private final TimeRangeTracker timeRangeTracker; private final TimeRangeTracker timeRangeTracker;
private final KeyValueScanner scanner; private final KeyValueScanner scanner;
private final boolean tagsPresent; private final boolean tagsPresent;
@ -36,7 +37,8 @@ public class MemStoreSnapshot {
public MemStoreSnapshot(long id, ImmutableSegment snapshot) { public MemStoreSnapshot(long id, ImmutableSegment snapshot) {
this.id = id; this.id = id;
this.cellsCount = snapshot.getCellsCount(); this.cellsCount = snapshot.getCellsCount();
this.size = snapshot.keySize(); this.dataSize = snapshot.keySize();
this.heapOverhead = snapshot.heapOverhead();
this.timeRangeTracker = snapshot.getTimeRangeTracker(); this.timeRangeTracker = snapshot.getTimeRangeTracker();
this.scanner = snapshot.getKeyValueScanner(); this.scanner = snapshot.getKeyValueScanner();
this.tagsPresent = snapshot.isTagsPresent(); this.tagsPresent = snapshot.isTagsPresent();
@ -59,8 +61,12 @@ public class MemStoreSnapshot {
/** /**
* @return Total memory size occupied by this snapshot. * @return Total memory size occupied by this snapshot.
*/ */
public long getSize() { public long getDataSize() {
return size; return dataSize;
}
public long getHeapOverhead() {
return this.heapOverhead;
} }
/** /**

View File

@ -0,0 +1,91 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Wraps the data size part and heap overhead of the memstore.
*/
@InterfaceAudience.Private
public class MemstoreSize {
static final MemstoreSize EMPTY_SIZE = new MemstoreSize();
private long dataSize;
private long heapOverhead;
public MemstoreSize() {
dataSize = 0;
heapOverhead = 0;
}
public MemstoreSize(long dataSize, long heapOverhead) {
this.dataSize = dataSize;
this.heapOverhead = heapOverhead;
}
public void incMemstoreSize(long dataSize, long heapOverhead) {
this.dataSize += dataSize;
this.heapOverhead += heapOverhead;
}
public void incMemstoreSize(MemstoreSize size) {
this.dataSize += size.dataSize;
this.heapOverhead += size.heapOverhead;
}
public void decMemstoreSize(long dataSize, long heapOverhead) {
this.dataSize -= dataSize;
this.heapOverhead -= heapOverhead;
}
public void decMemstoreSize(MemstoreSize size) {
this.dataSize -= size.dataSize;
this.heapOverhead -= size.heapOverhead;
}
public long getDataSize() {
return dataSize;
}
public long getHeapOverhead() {
return heapOverhead;
}
@Override
public boolean equals(Object obj) {
if (obj == null || !(obj instanceof MemstoreSize)) {
return false;
}
MemstoreSize other = (MemstoreSize) obj;
return this.dataSize == other.dataSize && this.heapOverhead == other.heapOverhead;
}
@Override
public int hashCode() {
long h = 13 * this.dataSize;
h = h + 14 * this.heapOverhead;
return (int) h;
}
@Override
public String toString() {
return "dataSize=" + this.dataSize + " , heapOverhead=" + this.heapOverhead;
}
}

View File

@ -685,7 +685,7 @@ class MetricsRegionServerWrapperImpl
tempNumStores += storeList.size(); tempNumStores += storeList.size();
for (Store store : storeList) { for (Store store : storeList) {
tempNumStoreFiles += store.getStorefilesCount(); tempNumStoreFiles += store.getStorefilesCount();
tempMemstoreSize += store.getMemStoreSize(); tempMemstoreSize += store.getSizeOfMemStore().getDataSize();
tempStoreFileSize += store.getStorefilesSize(); tempStoreFileSize += store.getStorefilesSize();
long storeMaxStoreFileAge = store.getMaxStoreFileAge(); long storeMaxStoreFileAge = store.getMaxStoreFileAge();

View File

@ -203,7 +203,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
if (region.stores != null) { if (region.stores != null) {
for (Store store : region.stores.values()) { for (Store store : region.stores.values()) {
tempNumStoreFiles += store.getStorefilesCount(); tempNumStoreFiles += store.getStorefilesCount();
tempMemstoreSize += store.getMemStoreSize(); tempMemstoreSize += store.getSizeOfMemStore().getDataSize();
tempStoreFileSize += store.getStorefilesSize(); tempStoreFileSize += store.getStorefilesSize();
long storeMaxStoreFileAge = store.getMaxStoreFileAge(); long storeMaxStoreFileAge = store.getMaxStoreFileAge();

View File

@ -18,8 +18,14 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.util.Iterator;
import java.util.SortedSet;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.ClassSize; import org.apache.hadoop.hbase.util.ClassSize;
@ -42,10 +48,60 @@ public class MutableSegment extends Segment {
* Adds the given cell into the segment * Adds the given cell into the segment
* @param cell the cell to add * @param cell the cell to add
* @param mslabUsed whether using MSLAB * @param mslabUsed whether using MSLAB
* @return the change in the heap size * @param memstoreSize
*/ */
public long add(Cell cell, boolean mslabUsed) { public void add(Cell cell, boolean mslabUsed, MemstoreSize memstoreSize) {
return internalAdd(cell, mslabUsed); internalAdd(cell, mslabUsed, memstoreSize);
}
public void upsert(Cell cell, long readpoint, MemstoreSize memstoreSize) {
internalAdd(cell, false, memstoreSize);
// Get the Cells for the row/family/qualifier regardless of timestamp.
// For this case we want to clean up any other puts
Cell firstCell = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
SortedSet<Cell> ss = this.tailSet(firstCell);
Iterator<Cell> it = ss.iterator();
// versions visible to oldest scanner
int versionsVisible = 0;
while (it.hasNext()) {
Cell cur = it.next();
if (cell == cur) {
// ignore the one just put in
continue;
}
// check that this is the row and column we are interested in, otherwise bail
if (CellUtil.matchingRows(cell, cur) && CellUtil.matchingQualifier(cell, cur)) {
// only remove Puts that concurrent scanners cannot possibly see
if (cur.getTypeByte() == KeyValue.Type.Put.getCode() && cur.getSequenceId() <= readpoint) {
if (versionsVisible >= 1) {
// if we get here we have seen at least one version visible to the oldest scanner,
// which means we can prove that no scanner will see this version
// false means there was a change, so give us the size.
// TODO when the removed cell ie.'cur' having its data in MSLAB, we can not release that
// area. Only the Cell object as such going way. We need to consider cellLen to be
// decreased there as 0 only. Just keeping it as existing code now. We need to know the
// removed cell is from MSLAB or not. Will do once HBASE-16438 is in
int cellLen = getCellLength(cur);
long heapOverheadDelta = heapOverheadChange(cur, true);
this.incSize(-cellLen, -heapOverheadDelta);
if (memstoreSize != null) {
memstoreSize.decMemstoreSize(cellLen, heapOverheadDelta);
}
it.remove();
} else {
versionsVisible++;
}
}
} else {
// past the row or column, done
break;
}
}
} }
/** /**
@ -67,9 +123,4 @@ public class MutableSegment extends Segment {
public long getMinTimestamp() { public long getMinTimestamp() {
return this.timeRangeTracker.getMin(); return this.timeRangeTracker.getMin();
} }
@Override
public long size() {
return keySize() + DEEP_OVERHEAD;
}
} }

View File

@ -195,7 +195,11 @@ public interface Region extends ConfigurationObserver {
*/ */
void updateWriteRequestsCount(long i); void updateWriteRequestsCount(long i);
/** @return memstore size for this region, in bytes */ /**
* @return memstore size for this region, in bytes. It just accounts data size of cells added to
* the memstores of this Region. Means size in bytes for key, value and tags within Cells.
* It wont consider any java heap overhead for the cell objects or any other.
*/
long getMemstoreSize(); long getMemstoreSize();
/** @return store services for this region, to access services required by store level needs */ /** @return store services for this region, to access services required by store level needs */

View File

@ -32,27 +32,37 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private @InterfaceAudience.Private
public class RegionServerAccounting { public class RegionServerAccounting {
private final AtomicLong atomicGlobalMemstoreSize = new AtomicLong(0); private final AtomicLong globalMemstoreDataSize = new AtomicLong(0);
private final AtomicLong globalMemstoreHeapOverhead = new AtomicLong(0);
// Store the edits size during replaying WAL. Use this to roll back the // Store the edits size during replaying WAL. Use this to roll back the
// global memstore size once a region opening failed. // global memstore size once a region opening failed.
private final ConcurrentMap<byte[], AtomicLong> replayEditsPerRegion = private final ConcurrentMap<byte[], MemstoreSize> replayEditsPerRegion =
new ConcurrentSkipListMap<byte[], AtomicLong>(Bytes.BYTES_COMPARATOR); new ConcurrentSkipListMap<byte[], MemstoreSize>(Bytes.BYTES_COMPARATOR);
/** /**
* @return the global Memstore size in the RegionServer * @return the global Memstore size in the RegionServer
*/ */
public long getGlobalMemstoreSize() { public long getGlobalMemstoreSize() {
return atomicGlobalMemstoreSize.get(); return globalMemstoreDataSize.get();
}
public long getGlobalMemstoreHeapOverhead() {
return this.globalMemstoreHeapOverhead.get();
} }
/** /**
* @param memStoreSize the Memstore size will be added to * @param memStoreSize the Memstore size will be added to
* the global Memstore size * the global Memstore size
* @return the global Memstore size in the RegionServer
*/ */
public long addAndGetGlobalMemstoreSize(long memStoreSize) { public void incGlobalMemstoreSize(MemstoreSize memStoreSize) {
return atomicGlobalMemstoreSize.addAndGet(memStoreSize); globalMemstoreDataSize.addAndGet(memStoreSize.getDataSize());
globalMemstoreHeapOverhead.addAndGet(memStoreSize.getHeapOverhead());
}
public void decGlobalMemstoreSize(MemstoreSize memStoreSize) {
globalMemstoreDataSize.addAndGet(-memStoreSize.getDataSize());
globalMemstoreHeapOverhead.addAndGet(-memStoreSize.getHeapOverhead());
} }
/*** /***
@ -60,15 +70,19 @@ public class RegionServerAccounting {
* *
* @param regionName region name. * @param regionName region name.
* @param memStoreSize the Memstore size will be added to replayEditsPerRegion. * @param memStoreSize the Memstore size will be added to replayEditsPerRegion.
* @return the replay edits size for the region.
*/ */
public long addAndGetRegionReplayEditsSize(byte[] regionName, long memStoreSize) { public void addRegionReplayEditsSize(byte[] regionName, MemstoreSize memStoreSize) {
AtomicLong replayEdistsSize = replayEditsPerRegion.get(regionName); MemstoreSize replayEdistsSize = replayEditsPerRegion.get(regionName);
// All ops on the same MemstoreSize object is going to be done by single thread, sequentially
// only. First calls to this method to increment the per region reply edits size and then call
// to either rollbackRegionReplayEditsSize or clearRegionReplayEditsSize as per the result of
// the region open operation. No need to handle multi thread issues on one region's entry in
// this Map.
if (replayEdistsSize == null) { if (replayEdistsSize == null) {
replayEdistsSize = new AtomicLong(0); replayEdistsSize = new MemstoreSize();
replayEditsPerRegion.put(regionName, replayEdistsSize); replayEditsPerRegion.put(regionName, replayEdistsSize);
} }
return replayEdistsSize.addAndGet(memStoreSize); replayEdistsSize.incMemstoreSize(memStoreSize);
} }
/** /**
@ -76,16 +90,13 @@ public class RegionServerAccounting {
* can't be opened. * can't be opened.
* *
* @param regionName the region which could not open. * @param regionName the region which could not open.
* @return the global Memstore size in the RegionServer
*/ */
public long rollbackRegionReplayEditsSize(byte[] regionName) { public void rollbackRegionReplayEditsSize(byte[] regionName) {
AtomicLong replayEditsSize = replayEditsPerRegion.get(regionName); MemstoreSize replayEditsSize = replayEditsPerRegion.get(regionName);
long editsSizeLong = 0L;
if (replayEditsSize != null) { if (replayEditsSize != null) {
editsSizeLong = -replayEditsSize.get();
clearRegionReplayEditsSize(regionName); clearRegionReplayEditsSize(regionName);
decGlobalMemstoreSize(replayEditsSize);
} }
return addAndGetGlobalMemstoreSize(editsSizeLong);
} }
/** /**
@ -96,5 +107,4 @@ public class RegionServerAccounting {
public void clearRegionReplayEditsSize(byte[] regionName) { public void clearRegionReplayEditsSize(byte[] regionName) {
replayEditsPerRegion.remove(regionName); replayEditsPerRegion.remove(regionName);
} }
} }

View File

@ -25,8 +25,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.util.StealJobQueue;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.wal.WAL;
/** /**
@ -37,7 +35,6 @@ import org.apache.hadoop.hbase.wal.WAL;
* take occasional lock and update size counters at the region level. * take occasional lock and update size counters at the region level.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@InterfaceStability.Evolving
public class RegionServicesForStores { public class RegionServicesForStores {
private static final int POOL_SIZE = 10; private static final int POOL_SIZE = 10;
@ -68,8 +65,8 @@ public class RegionServicesForStores {
region.unblockUpdates(); region.unblockUpdates();
} }
public long addAndGetGlobalMemstoreSize(long size) { public void addMemstoreSize(MemstoreSize size) {
return region.addAndGetGlobalMemstoreSize(size); region.addAndGetMemstoreSize(size);
} }
public HRegionInfo getRegionInfo() { public HRegionInfo getRegionInfo() {
@ -91,7 +88,7 @@ public class RegionServicesForStores {
} }
// methods for tests // methods for tests
long getGlobalMemstoreTotalSize() { long getMemstoreSize() {
return region.getMemstoreSize(); return region.getMemstoreSize();
} }
} }

View File

@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@ -55,11 +55,12 @@ public abstract class Segment {
private AtomicReference<CellSet> cellSet= new AtomicReference<CellSet>(); private AtomicReference<CellSet> cellSet= new AtomicReference<CellSet>();
private final CellComparator comparator; private final CellComparator comparator;
private long minSequenceId; protected long minSequenceId;
private MemStoreLAB memStoreLAB; private MemStoreLAB memStoreLAB;
// Sum of sizes of all Cells added to this Segment. Cell's heapSize is considered. This is not // Sum of sizes of all Cells added to this Segment. Cell's heapSize is considered. This is not
// including the heap overhead of this class. // including the heap overhead of this class.
protected final AtomicLong size; protected final AtomicLong dataSize;
protected final AtomicLong heapOverhead;
protected final TimeRangeTracker timeRangeTracker; protected final TimeRangeTracker timeRangeTracker;
protected volatile boolean tagsPresent; protected volatile boolean tagsPresent;
@ -69,7 +70,8 @@ public abstract class Segment {
this.comparator = comparator; this.comparator = comparator;
this.minSequenceId = Long.MAX_VALUE; this.minSequenceId = Long.MAX_VALUE;
this.memStoreLAB = memStoreLAB; this.memStoreLAB = memStoreLAB;
this.size = new AtomicLong(0); this.dataSize = new AtomicLong(0);
this.heapOverhead = new AtomicLong(0);
this.tagsPresent = false; this.tagsPresent = false;
this.timeRangeTracker = new TimeRangeTracker(); this.timeRangeTracker = new TimeRangeTracker();
} }
@ -79,7 +81,8 @@ public abstract class Segment {
this.comparator = segment.getComparator(); this.comparator = segment.getComparator();
this.minSequenceId = segment.getMinSequenceId(); this.minSequenceId = segment.getMinSequenceId();
this.memStoreLAB = segment.getMemStoreLAB(); this.memStoreLAB = segment.getMemStoreLAB();
this.size = new AtomicLong(segment.keySize()); this.dataSize = new AtomicLong(segment.keySize());
this.heapOverhead = new AtomicLong(segment.heapOverhead.get());
this.tagsPresent = segment.isTagsPresent(); this.tagsPresent = segment.isTagsPresent();
this.timeRangeTracker = segment.getTimeRangeTracker(); this.timeRangeTracker = segment.getTimeRangeTracker();
} }
@ -154,7 +157,7 @@ public abstract class Segment {
* Get cell length after serialized in {@link KeyValue} * Get cell length after serialized in {@link KeyValue}
*/ */
@VisibleForTesting @VisibleForTesting
int getCellLength(Cell cell) { static int getCellLength(Cell cell) {
return KeyValueUtil.length(cell); return KeyValueUtil.length(cell);
} }
@ -193,19 +196,26 @@ public abstract class Segment {
* @return Sum of all cell's size. * @return Sum of all cell's size.
*/ */
public long keySize() { public long keySize() {
return this.size.get(); return this.dataSize.get();
} }
/** /**
* @return the heap size of the segment * @return The heap overhead of this segment.
*/ */
public abstract long size(); public long heapOverhead() {
return this.heapOverhead.get();
}
/** /**
* Updates the heap size counter of the segment by the given delta * Updates the heap size counter of the segment by the given delta
*/ */
public void incSize(long delta) { protected void incSize(long delta, long heapOverhead) {
this.size.addAndGet(delta); this.dataSize.addAndGet(delta);
this.heapOverhead.addAndGet(heapOverhead);
}
protected void incHeapOverheadSize(long delta) {
this.heapOverhead.addAndGet(delta);
} }
public long getMinSequenceId() { public long getMinSequenceId() {
@ -252,36 +262,47 @@ public abstract class Segment {
return comparator; return comparator;
} }
protected long internalAdd(Cell cell, boolean mslabUsed) { protected void internalAdd(Cell cell, boolean mslabUsed, MemstoreSize memstoreSize) {
boolean succ = getCellSet().add(cell); boolean succ = getCellSet().add(cell);
long s = updateMetaInfo(cell, succ, mslabUsed); updateMetaInfo(cell, succ, mslabUsed, memstoreSize);
return s;
} }
protected long updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed) { protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed,
long s = heapSizeChange(cellToAdd, succ); MemstoreSize memstoreSize) {
long cellSize = 0;
// If there's already a same cell in the CellSet and we are using MSLAB, we must count in the // If there's already a same cell in the CellSet and we are using MSLAB, we must count in the
// MSLAB allocation size as well, or else there will be memory leak (occupied heap size larger // MSLAB allocation size as well, or else there will be memory leak (occupied heap size larger
// than the counted number) // than the counted number)
if (!succ && mslabUsed) { if (succ || mslabUsed) {
s += getCellLength(cellToAdd); cellSize = getCellLength(cellToAdd);
}
long overhead = heapOverheadChange(cellToAdd, succ);
incSize(cellSize, overhead);
if (memstoreSize != null) {
memstoreSize.incMemstoreSize(cellSize, overhead);
} }
getTimeRangeTracker().includeTimestamp(cellToAdd); getTimeRangeTracker().includeTimestamp(cellToAdd);
incSize(s);
minSequenceId = Math.min(minSequenceId, cellToAdd.getSequenceId()); minSequenceId = Math.min(minSequenceId, cellToAdd.getSequenceId());
// In no tags case this NoTagsKeyValue.getTagsLength() is a cheap call. // In no tags case this NoTagsKeyValue.getTagsLength() is a cheap call.
// When we use ACL CP or Visibility CP which deals with Tags during // When we use ACL CP or Visibility CP which deals with Tags during
// mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not // mutation, the TagRewriteCell.getTagsLength() is a cheaper call. We do not
// parse the byte[] to identify the tags length. // parse the byte[] to identify the tags length.
if( cellToAdd.getTagsLength() > 0) { if (cellToAdd.getTagsLength() > 0) {
tagsPresent = true; tagsPresent = true;
} }
return s;
} }
protected long heapSizeChange(Cell cell, boolean succ) { protected long heapOverheadChange(Cell cell, boolean succ) {
return succ ? ClassSize if (succ) {
.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + CellUtil.estimatedHeapSizeOf(cell)) : 0; if (cell instanceof ExtendedCell) {
return ClassSize
.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ((ExtendedCell) cell).heapOverhead());
}
// All cells in server side will be of type ExtendedCell. If not just go with estimation on
// the heap overhead considering it is KeyValue.
return ClassSize.align(ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + KeyValue.FIXED_OVERHEAD);
}
return 0;
} }
/** /**
@ -314,7 +335,7 @@ public abstract class Segment {
res += "isEmpty "+(isEmpty()?"yes":"no")+"; "; res += "isEmpty "+(isEmpty()?"yes":"no")+"; ";
res += "cellCount "+getCellsCount()+"; "; res += "cellCount "+getCellsCount()+"; ";
res += "cellsSize "+keySize()+"; "; res += "cellsSize "+keySize()+"; ";
res += "heapSize "+size()+"; "; res += "heapOverhead "+heapOverhead()+"; ";
res += "Min ts "+getMinTimestamp()+"; "; res += "Min ts "+getMinTimestamp()+"; ";
return res; return res;
} }

View File

@ -255,22 +255,45 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
/** /**
* @return The size of this store's memstore, in bytes * @return The size of this store's memstore, in bytes
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeOfMemStore()} instead.
*/ */
@Deprecated
long getMemStoreSize(); long getMemStoreSize();
/**
* @return The size of this store's memstore.
*/
MemstoreSize getSizeOfMemStore();
/** /**
* @return The amount of memory we could flush from this memstore; usually this is equal to * @return The amount of memory we could flush from this memstore; usually this is equal to
* {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of * {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of
* outstanding snapshots. * outstanding snapshots.
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeToFlush()} instead.
*/ */
@Deprecated
long getFlushableSize(); long getFlushableSize();
/**
* @return The amount of memory we could flush from this memstore; usually this is equal to
* {@link #getSizeOfMemStore()} unless we are carrying snapshots and then it will be the size of
* outstanding snapshots.
*/
MemstoreSize getSizeToFlush();
/** /**
* Returns the memstore snapshot size * Returns the memstore snapshot size
* @return size of the memstore snapshot * @return size of the memstore snapshot
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeOfSnapshot()} instead.
*/ */
@Deprecated
long getSnapshotSize(); long getSnapshotSize();
/**
* @return size of the memstore snapshot
*/
MemstoreSize getSizeOfSnapshot();
HColumnDescriptor getFamily(); HColumnDescriptor getFamily();
/** /**

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.MemstoreSize;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -61,7 +62,7 @@ public class TestClientPushback {
private static final TableName tableName = TableName.valueOf("client-pushback"); private static final TableName tableName = TableName.valueOf("client-pushback");
private static final byte[] family = Bytes.toBytes("f"); private static final byte[] family = Bytes.toBytes("f");
private static final byte[] qualifier = Bytes.toBytes("q"); private static final byte[] qualifier = Bytes.toBytes("q");
private static final long flushSizeBytes = 1024; private static final long flushSizeBytes = 256;
@BeforeClass @BeforeClass
public static void setupCluster() throws Exception{ public static void setupCluster() throws Exception{
@ -103,7 +104,8 @@ public class TestClientPushback {
table.put(p); table.put(p);
// get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data
int load = (int)((((HRegion)region).addAndGetGlobalMemstoreSize(0) * 100) / flushSizeBytes); int load = (int) ((((HRegion) region).addAndGetMemstoreSize(new MemstoreSize(0, 0)) * 100)
/ flushSizeBytes);
LOG.debug("Done writing some data to "+tableName); LOG.debug("Done writing some data to "+tableName);
// get the stats for the region hosting our table // get the stats for the region hosting our table

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.lang.management.ManagementFactory;
import java.lang.management.MemoryMXBean;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
@ -40,6 +38,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdge; import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
@ -121,8 +120,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
final KeyValue kv2 = new KeyValue(two, f, q, 10, v); final KeyValue kv2 = new KeyValue(two, f, q, 10, v);
// use case 1: both kvs in kvset // use case 1: both kvs in kvset
this.memstore.add(kv1.clone()); this.memstore.add(kv1.clone(), null);
this.memstore.add(kv2.clone()); this.memstore.add(kv2.clone(), null);
verifyScanAcrossSnapshot2(kv1, kv2); verifyScanAcrossSnapshot2(kv1, kv2);
// use case 2: both kvs in snapshot // use case 2: both kvs in snapshot
@ -132,12 +131,12 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
// use case 3: first in snapshot second in kvset // use case 3: first in snapshot second in kvset
this.memstore = new CompactingMemStore(HBaseConfiguration.create(), this.memstore = new CompactingMemStore(HBaseConfiguration.create(),
CellComparator.COMPARATOR, store, regionServicesForStores); CellComparator.COMPARATOR, store, regionServicesForStores);
this.memstore.add(kv1.clone()); this.memstore.add(kv1.clone(), null);
// As compaction is starting in the background the repetition // As compaction is starting in the background the repetition
// of the k1 might be removed BUT the scanners created earlier // of the k1 might be removed BUT the scanners created earlier
// should look on the OLD MutableCellSetSegment, so this should be OK... // should look on the OLD MutableCellSetSegment, so this should be OK...
this.memstore.snapshot(); this.memstore.snapshot();
this.memstore.add(kv2.clone()); this.memstore.add(kv2.clone(), null);
verifyScanAcrossSnapshot2(kv1,kv2); verifyScanAcrossSnapshot2(kv1,kv2);
} }
@ -173,7 +172,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
Thread.sleep(1); Thread.sleep(1);
addRows(this.memstore); addRows(this.memstore);
Cell closestToEmpty = ((CompactingMemStore)this.memstore).getNextRow(KeyValue.LOWESTKEY); Cell closestToEmpty = ((CompactingMemStore)this.memstore).getNextRow(KeyValue.LOWESTKEY);
assertTrue(KeyValue.COMPARATOR.compareRows(closestToEmpty, assertTrue(CellComparator.COMPARATOR.compareRows(closestToEmpty,
new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0); new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
for (int i = 0; i < ROW_COUNT; i++) { for (int i = 0; i < ROW_COUNT; i++) {
Cell nr = ((CompactingMemStore)this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i), Cell nr = ((CompactingMemStore)this.memstore).getNextRow(new KeyValue(Bytes.toBytes(i),
@ -181,7 +180,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
if (i + 1 == ROW_COUNT) { if (i + 1 == ROW_COUNT) {
assertEquals(nr, null); assertEquals(nr, null);
} else { } else {
assertTrue(KeyValue.COMPARATOR.compareRows(nr, assertTrue(CellComparator.COMPARATOR.compareRows(nr,
new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0); new KeyValue(Bytes.toBytes(i + 1), System.currentTimeMillis())) == 0);
} }
} }
@ -226,9 +225,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
byte[] val = Bytes.toBytes("testval"); byte[] val = Bytes.toBytes("testval");
//Setting up memstore //Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
//Pushing to pipeline //Pushing to pipeline
((CompactingMemStore)memstore).flushInMemory(); ((CompactingMemStore)memstore).flushInMemory();
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
@ -237,57 +236,11 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
assertEquals(3, memstore.getSnapshot().getCellsCount()); assertEquals(3, memstore.getSnapshot().getCellsCount());
//Adding value to "new" memstore //Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf4, val)); memstore.add(new KeyValue(row, fam, qf4, val), null);
memstore.add(new KeyValue(row, fam, qf5, val)); memstore.add(new KeyValue(row, fam, qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
} }
////////////////////////////////////
//Test for upsert with MSLAB
////////////////////////////////////
/**
* Test a pathological pattern that shows why we can't currently
* use the MSLAB for upsert workloads. This test inserts data
* in the following pattern:
*
* - row0001 through row1000 (fills up one 2M Chunk)
* - row0002 through row1001 (fills up another 2M chunk, leaves one reference
* to the first chunk
* - row0003 through row1002 (another chunk, another dangling reference)
*
* This causes OOME pretty quickly if we use MSLAB for upsert
* since each 2M chunk is held onto by a single reference.
*/
@Override
@Test
public void testUpsertMSLAB() throws Exception {
int ROW_SIZE = 2048;
byte[] qualifier = new byte[ROW_SIZE - 4];
MemoryMXBean bean = ManagementFactory.getMemoryMXBean();
for (int i = 0; i < 3; i++) { System.gc(); }
long usageBefore = bean.getHeapMemoryUsage().getUsed();
long size = 0;
long ts=0;
for (int newValue = 0; newValue < 1000; newValue++) {
for (int row = newValue; row < newValue + 1000; row++) {
byte[] rowBytes = Bytes.toBytes(row);
size += memstore.updateColumnValue(rowBytes, FAMILY, qualifier, newValue, ++ts);
}
}
System.out.println("Wrote " + ts + " vals");
for (int i = 0; i < 3; i++) { System.gc(); }
long usageAfter = bean.getHeapMemoryUsage().getUsed();
System.out.println("Memory used: " + (usageAfter - usageBefore)
+ " (heapsize: " + memstore.heapSize() +
" size: " + size + ")");
}
//////////////////////////////////// ////////////////////////////////////
// Test for periodic memstore flushes // Test for periodic memstore flushes
// based on time of oldest edit // based on time of oldest edit
@ -302,7 +255,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
@Override @Override
@Test @Test
public void testUpsertMemstoreSize() throws Exception { public void testUpsertMemstoreSize() throws Exception {
long oldSize = memstore.size(); MemstoreSize oldSize = memstore.size();
List<Cell> l = new ArrayList<Cell>(); List<Cell> l = new ArrayList<Cell>();
KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
@ -316,9 +269,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
l.add(kv2); l.add(kv2);
l.add(kv3); l.add(kv3);
this.memstore.upsert(l, 2);// readpoint is 2 this.memstore.upsert(l, 2, null);// readpoint is 2
long newSize = this.memstore.size(); MemstoreSize newSize = this.memstore.size();
assert (newSize > oldSize); assert (newSize.getDataSize() > oldSize.getDataSize());
//The kv1 should be removed. //The kv1 should be removed.
assert (memstore.getActive().getCellsCount() == 2); assert (memstore.getActive().getCellsCount() == 2);
@ -326,7 +279,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
kv4.setSequenceId(1); kv4.setSequenceId(1);
l.clear(); l.clear();
l.add(kv4); l.add(kv4);
this.memstore.upsert(l, 3); this.memstore.upsert(l, 3, null);
assertEquals(newSize, this.memstore.size()); assertEquals(newSize, this.memstore.size());
//The kv2 should be removed. //The kv2 should be removed.
assert (memstore.getActive().getCellsCount() == 2); assert (memstore.getActive().getCellsCount() == 2);
@ -348,7 +301,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
assertEquals(t, Long.MAX_VALUE); assertEquals(t, Long.MAX_VALUE);
// test the case that the timeOfOldestEdit is updated after a KV add // test the case that the timeOfOldestEdit is updated after a KV add
memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, "v")); memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, "v"), null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
// The method will also assert // The method will also assert
@ -356,7 +309,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
t = runSnapshot(memstore, true); t = runSnapshot(memstore, true);
// test the case that the timeOfOldestEdit is updated after a KV delete // test the case that the timeOfOldestEdit is updated after a KV delete
memstore.delete(KeyValueTestUtil.create("r", "f", "q", 100, "v")); memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, KeyValue.Type.Delete, "v"), null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
t = runSnapshot(memstore, true); t = runSnapshot(memstore, true);
@ -366,7 +319,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
kv1.setSequenceId(100); kv1.setSequenceId(100);
l.add(kv1); l.add(kv1);
memstore.upsert(l, 1000); memstore.upsert(l, 1000, null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
} finally { } finally {
@ -384,7 +337,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
MemStoreSnapshot snapshot = hmc.snapshot(); MemStoreSnapshot snapshot = hmc.snapshot();
if (useForce) { if (useForce) {
// Make some assertions about what just happened. // Make some assertions about what just happened.
assertTrue("History size has not increased", oldHistorySize < snapshot.getSize()); assertTrue("History size has not increased", oldHistorySize < snapshot.getDataSize());
long t = hmc.timeOfOldestEdit(); long t = hmc.timeOfOldestEdit();
assertTrue("Time of oldest edit is not Long.MAX_VALUE", t == Long.MAX_VALUE); assertTrue("Time of oldest edit is not Long.MAX_VALUE", t == Long.MAX_VALUE);
hmc.clearSnapshot(snapshot.getId()); hmc.clearSnapshot(snapshot.getId());
@ -421,9 +374,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
byte[] val = Bytes.toBytes("testval"); byte[] val = Bytes.toBytes("testval");
// Setting up memstore // Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
// Creating a snapshot // Creating a snapshot
MemStoreSnapshot snapshot = memstore.snapshot(); MemStoreSnapshot snapshot = memstore.snapshot();
@ -431,8 +384,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf4, val)); memstore.add(new KeyValue(row, fam, qf4, val), null);
memstore.add(new KeyValue(row, fam, qf5, val)); memstore.add(new KeyValue(row, fam, qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
@ -456,9 +409,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
byte[] val = Bytes.toBytes("testval"); byte[] val = Bytes.toBytes("testval");
// Setting up memstore // Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
// Creating a snapshot // Creating a snapshot
MemStoreSnapshot snapshot = memstore.snapshot(); MemStoreSnapshot snapshot = memstore.snapshot();
@ -466,8 +419,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf4, val)); memstore.add(new KeyValue(row, fam, qf4, val), null);
memstore.add(new KeyValue(row, fam, qf5, val)); memstore.add(new KeyValue(row, fam, qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
// opening scanner before clear the snapshot // opening scanner before clear the snapshot
@ -491,8 +444,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
snapshot = memstore.snapshot(); snapshot = memstore.snapshot();
// Adding more value // Adding more value
memstore.add(new KeyValue(row, fam, qf6, val)); memstore.add(new KeyValue(row, fam, qf6, val), null);
memstore.add(new KeyValue(row, fam, qf7, val)); memstore.add(new KeyValue(row, fam, qf7, val), null);
// opening scanners // opening scanners
scanners = memstore.getScanners(0); scanners = memstore.getScanners(0);
// close scanners before clear the snapshot // close scanners before clear the snapshot
@ -521,9 +474,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
byte[] val = Bytes.toBytes("testval"); byte[] val = Bytes.toBytes("testval");
// Setting up memstore // Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, 1, val)); memstore.add(new KeyValue(row, fam, qf1, 1, val), null);
memstore.add(new KeyValue(row, fam, qf2, 1, val)); memstore.add(new KeyValue(row, fam, qf2, 1, val), null);
memstore.add(new KeyValue(row, fam, qf3, 1, val)); memstore.add(new KeyValue(row, fam, qf3, 1, val), null);
// Creating a pipeline // Creating a pipeline
((CompactingMemStore)memstore).disableCompaction(); ((CompactingMemStore)memstore).disableCompaction();
@ -531,8 +484,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf1, 2, val)); memstore.add(new KeyValue(row, fam, qf1, 2, val), null);
memstore.add(new KeyValue(row, fam, qf2, 2, val)); memstore.add(new KeyValue(row, fam, qf2, 2, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
// pipeline bucket 2 // pipeline bucket 2
@ -547,9 +500,9 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf3, 3, val)); memstore.add(new KeyValue(row, fam, qf3, 3, val), null);
memstore.add(new KeyValue(row, fam, qf2, 3, val)); memstore.add(new KeyValue(row, fam, qf2, 3, val), null);
memstore.add(new KeyValue(row, fam, qf1, 3, val)); memstore.add(new KeyValue(row, fam, qf1, 3, val), null);
assertEquals(3, memstore.getActive().getCellsCount()); assertEquals(3, memstore.getActive().getCellsCount());
assertTrue(chunkPool.getPoolSize() == 0); assertTrue(chunkPool.getPoolSize() == 0);
@ -570,8 +523,8 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
snapshot = memstore.snapshot(); snapshot = memstore.snapshot();
// Adding more value // Adding more value
memstore.add(new KeyValue(row, fam, qf2, 4, val)); memstore.add(new KeyValue(row, fam, qf2, 4, val), null);
memstore.add(new KeyValue(row, fam, qf3, 4, val)); memstore.add(new KeyValue(row, fam, qf3, 4, val), null);
// opening scanners // opening scanners
scanners = memstore.getScanners(0); scanners = memstore.getScanners(0);
// close scanners before clear the snapshot // close scanners before clear the snapshot
@ -597,20 +550,27 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
String[] keys1 = { "A", "A", "B", "C" }; //A1, A2, B3, C4 String[] keys1 = { "A", "A", "B", "C" }; //A1, A2, B3, C4
// test 1 bucket // test 1 bucket
addRowsByKeys(memstore, keys1); int totalCellsLen = addRowsByKeys(memstore, keys1);
assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize()); long totalHeapOverhead = 4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
long size = memstore.getFlushableSize(); MemstoreSize size = memstore.getFlushableSize();
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same size so adjusting
// totalCellsLen
totalCellsLen = (totalCellsLen * 3) / 4;
totalHeapOverhead = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(3, s.getCellsCount()); assertEquals(3, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
} }
@ -624,11 +584,13 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
String[] keys1 = { "A", "A", "B", "C" }; String[] keys1 = { "A", "A", "B", "C" };
String[] keys2 = { "A", "B", "D" }; String[] keys2 = { "A", "B", "D" };
addRowsByKeys(memstore, keys1); int totalCellsLen1 = addRowsByKeys(memstore, keys1);
long totalHeapOverhead = 4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
long size = memstore.getFlushableSize(); MemstoreSize size = memstore.getFlushableSize();
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
int counter = 0; int counter = 0;
for ( Segment s : memstore.getSegments()) { for ( Segment s : memstore.getSegments()) {
@ -636,22 +598,32 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
} }
assertEquals(3, counter); assertEquals(3, counter);
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same time so adjusting
// totalCellsLen
totalCellsLen1 = (totalCellsLen1 * 3) / 4;
assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
totalHeapOverhead = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
addRowsByKeys(memstore, keys2); int totalCellsLen2 = addRowsByKeys(memstore, keys2);
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize()); totalHeapOverhead += 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(368, regionServicesForStores.getGlobalMemstoreTotalSize()); totalCellsLen2 = totalCellsLen2 / 3;// 2 cells duplicated in set 2
assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
totalHeapOverhead = 4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(4, s.getCellsCount()); assertEquals(4, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
} }
@ -666,33 +638,47 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
String[] keys2 = { "A", "B", "D" }; String[] keys2 = { "A", "B", "D" };
String[] keys3 = { "D", "B", "B" }; String[] keys3 = { "D", "B", "B" };
addRowsByKeys(memstore, keys1); int totalCellsLen1 = addRowsByKeys(memstore, keys1);// Adding 4 cells.
assertEquals(496, region.getMemstoreSize()); assertEquals(totalCellsLen1, region.getMemstoreSize());
long totalHeapOverhead = 4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
long size = memstore.getFlushableSize(); MemstoreSize size = memstore.getFlushableSize();
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact
String tstStr = "\n\nFlushable size after first flush in memory:" + size
+ ". Is MemmStore in compaction?:" + ((CompactingMemStore)memstore).isMemStoreFlushingInMemory();
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same time so adjusting
// totalCellsLen
totalCellsLen1 = (totalCellsLen1 * 3) / 4;
assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
// In memory flush to make a CellArrayMap instead of CSLM. See the overhead diff.
totalHeapOverhead = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
addRowsByKeys(memstore, keys2); int totalCellsLen2 = addRowsByKeys(memstore, keys2);// Adding 3 more cells.
long totalHeapOverhead2 = 3
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
tstStr += " After adding second part of the keys. Memstore size: " + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
region.getMemstoreSize() + ", Memstore Total Size: " + assertEquals(totalHeapOverhead + totalHeapOverhead2,
regionServicesForStores.getGlobalMemstoreTotalSize() + "\n\n"; ((CompactingMemStore) memstore).heapOverhead());
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize()); ((CompactingMemStore) memstore).disableCompaction();
((CompactingMemStore)memstore).disableCompaction();
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline without compaction ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline without compaction
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize()); // No change in the cells data size. ie. memstore size. as there is no compaction.
assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead + totalHeapOverhead2,
((CompactingMemStore) memstore).heapOverhead());
addRowsByKeys(memstore, keys3); int totalCellsLen3 = addRowsByKeys(memstore, keys3);// 3 more cells added
assertEquals(1016, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3,
regionServicesForStores.getMemstoreSize());
long totalHeapOverhead3 = 3
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalHeapOverhead + totalHeapOverhead2 + totalHeapOverhead3,
((CompactingMemStore) memstore).heapOverhead());
((CompactingMemStore)memstore).enableCompaction(); ((CompactingMemStore)memstore).enableCompaction();
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
@ -701,34 +687,47 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
Threads.sleep(10); Threads.sleep(10);
} }
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(384, regionServicesForStores.getGlobalMemstoreTotalSize()); // active flushed to pipeline and all 3 segments compacted. Will get rid of duplicated cells.
// Out of total 10, only 4 cells are unique
totalCellsLen2 = totalCellsLen2 / 3;// 2 out of 3 cells are duplicated
totalCellsLen3 = 0;// All duplicated cells.
assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3,
regionServicesForStores.getMemstoreSize());
// Only 4 unique cells left
assertEquals(4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY),
((CompactingMemStore) memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(4, s.getCellsCount()); assertEquals(4, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
//assertTrue(tstStr, false); //assertTrue(tstStr, false);
} }
private void addRowsByKeys(final AbstractMemStore hmc, String[] keys) { private int addRowsByKeys(final AbstractMemStore hmc, String[] keys) {
byte[] fam = Bytes.toBytes("testfamily"); byte[] fam = Bytes.toBytes("testfamily");
byte[] qf = Bytes.toBytes("testqualifier"); byte[] qf = Bytes.toBytes("testqualifier");
long size = hmc.getActive().keySize(); long size = hmc.getActive().keySize();
long heapOverhead = hmc.getActive().heapOverhead();
int totalLen = 0;
for (int i = 0; i < keys.length; i++) { for (int i = 0; i < keys.length; i++) {
long timestamp = System.currentTimeMillis(); long timestamp = System.currentTimeMillis();
Threads.sleep(1); // to make sure each kv gets a different ts Threads.sleep(1); // to make sure each kv gets a different ts
byte[] row = Bytes.toBytes(keys[i]); byte[] row = Bytes.toBytes(keys[i]);
byte[] val = Bytes.toBytes(keys[i] + i); byte[] val = Bytes.toBytes(keys[i] + i);
KeyValue kv = new KeyValue(row, fam, qf, timestamp, val); KeyValue kv = new KeyValue(row, fam, qf, timestamp, val);
hmc.add(kv); totalLen += kv.getLength();
hmc.add(kv, null);
LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp()); LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp());
} }
regionServicesForStores.addAndGetGlobalMemstoreSize(hmc.getActive().keySize() - size); regionServicesForStores.addMemstoreSize(new MemstoreSize(hmc.getActive().keySize() - size,
hmc.getActive().heapOverhead() - heapOverhead));
return totalLen;
} }
private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge { private class EnvironmentEdgeForMemstoreTest implements EnvironmentEdge {

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -78,27 +78,34 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
String[] keys1 = { "A", "A", "B", "C" }; //A1, A2, B3, C4 String[] keys1 = { "A", "A", "B", "C" }; //A1, A2, B3, C4
// test 1 bucket // test 1 bucket
addRowsByKeys(memstore, keys1); long totalCellsLen = addRowsByKeys(memstore, keys1);
assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize()); long totalHeapOverhead = 4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
assertEquals(4, memstore.getActive().getCellsCount()); assertEquals(4, memstore.getActive().getCellsCount());
long size = memstore.getFlushableSize(); MemstoreSize size = memstore.getFlushableSize();
((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact
while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) { while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
Threads.sleep(10); Threads.sleep(10);
} }
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same size so adjusting
// totalCellsLen
totalCellsLen = (totalCellsLen * 3) / 4;
assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize());
totalHeapOverhead = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalHeapOverhead, ((CompactingMemStore)memstore).heapOverhead());
for ( Segment s : memstore.getSegments()) { for ( Segment s : memstore.getSegments()) {
counter += s.getCellsCount(); counter += s.getCellsCount();
} }
assertEquals(3, counter); assertEquals(3, counter);
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(3, s.getCellsCount()); assertEquals(3, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
} }
@ -108,13 +115,12 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
String[] keys1 = { "A", "A", "B", "C" }; String[] keys1 = { "A", "A", "B", "C" };
String[] keys2 = { "A", "B", "D" }; String[] keys2 = { "A", "B", "D" };
addRowsByKeys(memstore, keys1); long totalCellsLen1 = addRowsByKeys(memstore, keys1);
assertEquals(496, regionServicesForStores.getGlobalMemstoreTotalSize()); long totalHeapOverhead1 = 4
long size = memstore.getFlushableSize(); * (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
// assertTrue( assertEquals(totalHeapOverhead1, ((CompactingMemStore) memstore).heapOverhead());
// "\n\n<<< This is the active size with 4 keys - " + memstore.getActive().getSize() MemstoreSize size = memstore.getFlushableSize();
// + ". This is the memstore flushable size - " + size + "\n",false);
((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact
while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) { while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
@ -126,10 +132,19 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
} }
assertEquals(3,counter); assertEquals(3,counter);
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same size so adjusting
// totalCellsLen
totalCellsLen1 = (totalCellsLen1 * 3) / 4;
totalHeapOverhead1 = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead1, ((CompactingMemStore) memstore).heapOverhead());
addRowsByKeys(memstore, keys2); long totalCellsLen2 = addRowsByKeys(memstore, keys2);
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize()); long totalHeapOverhead2 = 3
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead1 + totalHeapOverhead2,
((CompactingMemStore) memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact
@ -147,14 +162,18 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
counter += s.getCellsCount(); counter += s.getCellsCount();
} }
assertEquals(4,counter); assertEquals(4,counter);
assertEquals(368, regionServicesForStores.getGlobalMemstoreTotalSize()); totalCellsLen2 = totalCellsLen2 / 3;// 2 cells duplicated in set 2
assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
totalHeapOverhead2 = 1 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalHeapOverhead1 + totalHeapOverhead2,
((CompactingMemStore) memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(4, s.getCellsCount()); assertEquals(4, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
} }
@ -165,36 +184,49 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
String[] keys2 = { "A", "B", "D" }; String[] keys2 = { "A", "B", "D" };
String[] keys3 = { "D", "B", "B" }; String[] keys3 = { "D", "B", "B" };
addRowsByKeys(memstore, keys1); long totalCellsLen1 = addRowsByKeys(memstore, keys1);
assertEquals(496, region.getMemstoreSize()); long totalHeapOverhead1 = 4
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen1, region.getMemstoreSize());
assertEquals(totalHeapOverhead1, ((CompactingMemStore) memstore).heapOverhead());
long size = memstore.getFlushableSize(); MemstoreSize size = memstore.getFlushableSize();
((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact
String tstStr = "\n\nFlushable size after first flush in memory:" + size + ". Is MemmStore in compaction?:"
+ ((CompactingMemStore) memstore).isMemStoreFlushingInMemory();
while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) { while (((CompactingMemStore) memstore).isMemStoreFlushingInMemory()) {
Threads.sleep(10); Threads.sleep(10);
} }
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(264, regionServicesForStores.getGlobalMemstoreTotalSize()); // One cell is duplicated and the compaction will remove it. All cells of same size so adjusting
// totalCellsLen
totalCellsLen1 = (totalCellsLen1 * 3) / 4;
totalHeapOverhead1 = 3 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY);
assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead1, ((CompactingMemStore) memstore).heapOverhead());
addRowsByKeys(memstore, keys2); long totalCellsLen2 = addRowsByKeys(memstore, keys2);
long totalHeapOverhead2 = 3
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
tstStr += " After adding second part of the keys. Memstore size: " + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
region.getMemstoreSize() + ", Memstore Total Size: " + assertEquals(totalHeapOverhead1 + totalHeapOverhead2,
regionServicesForStores.getGlobalMemstoreTotalSize() + "\n\n"; ((CompactingMemStore) memstore).heapOverhead());
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize());
((CompactingMemStore) memstore).disableCompaction(); ((CompactingMemStore) memstore).disableCompaction();
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline without compaction ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline without compaction
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(640, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead1 + totalHeapOverhead2,
((CompactingMemStore) memstore).heapOverhead());
addRowsByKeys(memstore, keys3); long totalCellsLen3 = addRowsByKeys(memstore, keys3);
assertEquals(1016, regionServicesForStores.getGlobalMemstoreTotalSize()); long totalHeapOverhead3 = 3
* (KeyValue.FIXED_OVERHEAD + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY);
assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3,
regionServicesForStores.getMemstoreSize());
assertEquals(totalHeapOverhead1 + totalHeapOverhead2 + totalHeapOverhead3,
((CompactingMemStore) memstore).heapOverhead());
((CompactingMemStore) memstore).enableCompaction(); ((CompactingMemStore) memstore).enableCompaction();
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
@ -203,14 +235,22 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
Threads.sleep(10); Threads.sleep(10);
} }
assertEquals(0, memstore.getSnapshot().getCellsCount()); assertEquals(0, memstore.getSnapshot().getCellsCount());
assertEquals(384, regionServicesForStores.getGlobalMemstoreTotalSize()); // active flushed to pipeline and all 3 segments compacted. Will get rid of duplicated cells.
// Out of total 10, only 4 cells are unique
totalCellsLen2 = totalCellsLen2 / 3;// 2 out of 3 cells are duplicated
totalCellsLen3 = 0;// All duplicated cells.
assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3,
regionServicesForStores.getMemstoreSize());
// Only 4 unique cells left
assertEquals(4 * (KeyValue.FIXED_OVERHEAD + ClassSize.CELL_ARRAY_MAP_ENTRY),
((CompactingMemStore) memstore).heapOverhead());
size = memstore.getFlushableSize(); size = memstore.getFlushableSize();
MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot
region.addAndGetGlobalMemstoreSize(-size); // simulate flusher region.decrMemstoreSize(size); // simulate flusher
ImmutableSegment s = memstore.getSnapshot(); ImmutableSegment s = memstore.getSnapshot();
assertEquals(4, s.getCellsCount()); assertEquals(4, s.getCellsCount());
assertEquals(0, regionServicesForStores.getGlobalMemstoreTotalSize()); assertEquals(0, regionServicesForStores.getMemstoreSize());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
@ -339,24 +379,25 @@ public class TestCompactingToCellArrayMapMemStore extends TestCompactingMemStore
byte[] qf = Bytes.toBytes("testqualifier"+j); byte[] qf = Bytes.toBytes("testqualifier"+j);
byte[] val = Bytes.toBytes(keys[i] + j); byte[] val = Bytes.toBytes(keys[i] + j);
KeyValue kv = new KeyValue(row, fam, qf, timestamp, val); KeyValue kv = new KeyValue(row, fam, qf, timestamp, val);
hmc.add(kv); hmc.add(kv, null);
} }
} }
} }
private void addRowsByKeys(final AbstractMemStore hmc, String[] keys) { private long addRowsByKeys(final AbstractMemStore hmc, String[] keys) {
byte[] fam = Bytes.toBytes("testfamily"); byte[] fam = Bytes.toBytes("testfamily");
byte[] qf = Bytes.toBytes("testqualifier"); byte[] qf = Bytes.toBytes("testqualifier");
long size = hmc.getActive().size();// MemstoreSize memstoreSize = new MemstoreSize();
for (int i = 0; i < keys.length; i++) { for (int i = 0; i < keys.length; i++) {
long timestamp = System.currentTimeMillis(); long timestamp = System.currentTimeMillis();
Threads.sleep(1); // to make sure each kv gets a different ts Threads.sleep(1); // to make sure each kv gets a different ts
byte[] row = Bytes.toBytes(keys[i]); byte[] row = Bytes.toBytes(keys[i]);
byte[] val = Bytes.toBytes(keys[i] + i); byte[] val = Bytes.toBytes(keys[i] + i);
KeyValue kv = new KeyValue(row, fam, qf, timestamp, val); KeyValue kv = new KeyValue(row, fam, qf, timestamp, val);
hmc.add(kv); hmc.add(kv, memstoreSize);
LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp()); LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp());
} }
regionServicesForStores.addAndGetGlobalMemstoreSize(hmc.getActive().size() - size);// regionServicesForStores.addMemstoreSize(memstoreSize);
return memstoreSize.getDataSize();
} }
} }

View File

@ -105,10 +105,10 @@ public class TestDefaultMemStore {
public void testPutSameKey() { public void testPutSameKey() {
byte[] bytes = Bytes.toBytes(getName()); byte[] bytes = Bytes.toBytes(getName());
KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes); KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes);
this.memstore.add(kv); this.memstore.add(kv, null);
byte[] other = Bytes.toBytes("somethingelse"); byte[] other = Bytes.toBytes("somethingelse");
KeyValue samekey = new KeyValue(bytes, bytes, bytes, other); KeyValue samekey = new KeyValue(bytes, bytes, bytes, other);
this.memstore.add(samekey); this.memstore.add(samekey, null);
Cell found = this.memstore.getActive().first(); Cell found = this.memstore.getActive().first();
assertEquals(1, this.memstore.getActive().getCellsCount()); assertEquals(1, this.memstore.getActive().getCellsCount());
assertTrue(Bytes.toString(found.getValueArray()), CellUtil.matchingValue(samekey, found)); assertTrue(Bytes.toString(found.getValueArray()), CellUtil.matchingValue(samekey, found));
@ -118,23 +118,28 @@ public class TestDefaultMemStore {
public void testPutSameCell() { public void testPutSameCell() {
byte[] bytes = Bytes.toBytes(getName()); byte[] bytes = Bytes.toBytes(getName());
KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes); KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes);
long sizeChangeForFirstCell = this.memstore.add(kv); MemstoreSize sizeChangeForFirstCell = new MemstoreSize();
long sizeChangeForSecondCell = this.memstore.add(kv); this.memstore.add(kv, sizeChangeForFirstCell);
MemstoreSize sizeChangeForSecondCell = new MemstoreSize();
this.memstore.add(kv, sizeChangeForSecondCell);
// make sure memstore size increase won't double-count MSLAB chunk size // make sure memstore size increase won't double-count MSLAB chunk size
assertEquals(AbstractMemStore.heapSizeChange(kv, true), sizeChangeForFirstCell); assertEquals(Segment.getCellLength(kv), sizeChangeForFirstCell.getDataSize());
assertEquals(this.memstore.active.heapOverheadChange(kv, true),
sizeChangeForFirstCell.getHeapOverhead());
Segment segment = this.memstore.getActive(); Segment segment = this.memstore.getActive();
MemStoreLAB msLab = segment.getMemStoreLAB(); MemStoreLAB msLab = segment.getMemStoreLAB();
if (msLab != null) { if (msLab != null) {
// make sure memstore size increased even when writing the same cell, if using MSLAB // make sure memstore size increased even when writing the same cell, if using MSLAB
assertEquals(segment.getCellLength(kv), sizeChangeForSecondCell); assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
// make sure chunk size increased even when writing the same cell, if using MSLAB // make sure chunk size increased even when writing the same cell, if using MSLAB
if (msLab instanceof HeapMemStoreLAB) { if (msLab instanceof HeapMemStoreLAB) {
assertEquals(2 * segment.getCellLength(kv), assertEquals(2 * Segment.getCellLength(kv),
((HeapMemStoreLAB) msLab).getCurrentChunk().getNextFreeOffset()); ((HeapMemStoreLAB) msLab).getCurrentChunk().getNextFreeOffset());
} }
} else { } else {
// make sure no memstore size change w/o MSLAB // make sure no memstore size change w/o MSLAB
assertEquals(0, sizeChangeForSecondCell); assertEquals(0, sizeChangeForSecondCell.getDataSize());
assertEquals(0, sizeChangeForSecondCell.getHeapOverhead());
} }
} }
@ -244,8 +249,8 @@ public class TestDefaultMemStore {
final KeyValue kv2 = new KeyValue(two, f, q, v); final KeyValue kv2 = new KeyValue(two, f, q, v);
// use case 1: both kvs in kvset // use case 1: both kvs in kvset
this.memstore.add(kv1.clone()); this.memstore.add(kv1.clone(), null);
this.memstore.add(kv2.clone()); this.memstore.add(kv2.clone(), null);
verifyScanAcrossSnapshot2(kv1, kv2); verifyScanAcrossSnapshot2(kv1, kv2);
// use case 2: both kvs in snapshot // use case 2: both kvs in snapshot
@ -254,9 +259,9 @@ public class TestDefaultMemStore {
// use case 3: first in snapshot second in kvset // use case 3: first in snapshot second in kvset
this.memstore = new DefaultMemStore(); this.memstore = new DefaultMemStore();
this.memstore.add(kv1.clone()); this.memstore.add(kv1.clone(), null);
this.memstore.snapshot(); this.memstore.snapshot();
this.memstore.add(kv2.clone()); this.memstore.add(kv2.clone(), null);
verifyScanAcrossSnapshot2(kv1, kv2); verifyScanAcrossSnapshot2(kv1, kv2);
} }
@ -302,7 +307,7 @@ public class TestDefaultMemStore {
KeyValue kv1 = new KeyValue(row, f, q1, v); KeyValue kv1 = new KeyValue(row, f, q1, v);
kv1.setSequenceId(w.getWriteNumber()); kv1.setSequenceId(w.getWriteNumber());
memstore.add(kv1); memstore.add(kv1, null);
KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); KeyValueScanner s = this.memstore.getScanners(mvcc.getReadPoint()).get(0);
assertScannerResults(s, new KeyValue[]{}); assertScannerResults(s, new KeyValue[]{});
@ -315,7 +320,7 @@ public class TestDefaultMemStore {
w = mvcc.begin(); w = mvcc.begin();
KeyValue kv2 = new KeyValue(row, f, q2, v); KeyValue kv2 = new KeyValue(row, f, q2, v);
kv2.setSequenceId(w.getWriteNumber()); kv2.setSequenceId(w.getWriteNumber());
memstore.add(kv2); memstore.add(kv2, null);
s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); s = this.memstore.getScanners(mvcc.getReadPoint()).get(0);
assertScannerResults(s, new KeyValue[]{kv1}); assertScannerResults(s, new KeyValue[]{kv1});
@ -347,11 +352,11 @@ public class TestDefaultMemStore {
KeyValue kv11 = new KeyValue(row, f, q1, v1); KeyValue kv11 = new KeyValue(row, f, q1, v1);
kv11.setSequenceId(w.getWriteNumber()); kv11.setSequenceId(w.getWriteNumber());
memstore.add(kv11); memstore.add(kv11, null);
KeyValue kv12 = new KeyValue(row, f, q2, v1); KeyValue kv12 = new KeyValue(row, f, q2, v1);
kv12.setSequenceId(w.getWriteNumber()); kv12.setSequenceId(w.getWriteNumber());
memstore.add(kv12); memstore.add(kv12, null);
mvcc.completeAndWait(w); mvcc.completeAndWait(w);
// BEFORE STARTING INSERT 2, SEE FIRST KVS // BEFORE STARTING INSERT 2, SEE FIRST KVS
@ -362,11 +367,11 @@ public class TestDefaultMemStore {
w = mvcc.begin(); w = mvcc.begin();
KeyValue kv21 = new KeyValue(row, f, q1, v2); KeyValue kv21 = new KeyValue(row, f, q1, v2);
kv21.setSequenceId(w.getWriteNumber()); kv21.setSequenceId(w.getWriteNumber());
memstore.add(kv21); memstore.add(kv21, null);
KeyValue kv22 = new KeyValue(row, f, q2, v2); KeyValue kv22 = new KeyValue(row, f, q2, v2);
kv22.setSequenceId(w.getWriteNumber()); kv22.setSequenceId(w.getWriteNumber());
memstore.add(kv22); memstore.add(kv22, null);
// BEFORE COMPLETING INSERT 2, SEE FIRST KVS // BEFORE COMPLETING INSERT 2, SEE FIRST KVS
s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); s = this.memstore.getScanners(mvcc.getReadPoint()).get(0);
@ -400,11 +405,11 @@ public class TestDefaultMemStore {
KeyValue kv11 = new KeyValue(row, f, q1, v1); KeyValue kv11 = new KeyValue(row, f, q1, v1);
kv11.setSequenceId(w.getWriteNumber()); kv11.setSequenceId(w.getWriteNumber());
memstore.add(kv11); memstore.add(kv11, null);
KeyValue kv12 = new KeyValue(row, f, q2, v1); KeyValue kv12 = new KeyValue(row, f, q2, v1);
kv12.setSequenceId(w.getWriteNumber()); kv12.setSequenceId(w.getWriteNumber());
memstore.add(kv12); memstore.add(kv12, null);
mvcc.completeAndWait(w); mvcc.completeAndWait(w);
// BEFORE STARTING INSERT 2, SEE FIRST KVS // BEFORE STARTING INSERT 2, SEE FIRST KVS
@ -416,7 +421,7 @@ public class TestDefaultMemStore {
KeyValue kvDel = new KeyValue(row, f, q2, kv11.getTimestamp(), KeyValue kvDel = new KeyValue(row, f, q2, kv11.getTimestamp(),
KeyValue.Type.DeleteColumn); KeyValue.Type.DeleteColumn);
kvDel.setSequenceId(w.getWriteNumber()); kvDel.setSequenceId(w.getWriteNumber());
memstore.add(kvDel); memstore.add(kvDel, null);
// BEFORE COMPLETING DELETE, SEE FIRST KVS // BEFORE COMPLETING DELETE, SEE FIRST KVS
s = this.memstore.getScanners(mvcc.getReadPoint()).get(0); s = this.memstore.getScanners(mvcc.getReadPoint()).get(0);
@ -478,7 +483,7 @@ public class TestDefaultMemStore {
KeyValue kv = new KeyValue(row, f, q1, i, v); KeyValue kv = new KeyValue(row, f, q1, i, v);
kv.setSequenceId(w.getWriteNumber()); kv.setSequenceId(w.getWriteNumber());
memstore.add(kv); memstore.add(kv, null);
mvcc.completeAndWait(w); mvcc.completeAndWait(w);
// Assert that we can read back // Assert that we can read back
@ -543,9 +548,9 @@ public class TestDefaultMemStore {
KeyValue key1 = new KeyValue(row, family, qf, stamps[1], values[1]); KeyValue key1 = new KeyValue(row, family, qf, stamps[1], values[1]);
KeyValue key2 = new KeyValue(row, family, qf, stamps[2], values[2]); KeyValue key2 = new KeyValue(row, family, qf, stamps[2], values[2]);
m.add(key0); m.add(key0, null);
m.add(key1); m.add(key1, null);
m.add(key2); m.add(key2, null);
assertTrue("Expected memstore to hold 3 values, actually has " + assertTrue("Expected memstore to hold 3 values, actually has " +
m.getActive().getCellsCount(), m.getActive().getCellsCount() == 3); m.getActive().getCellsCount(), m.getActive().getCellsCount() == 3);
@ -619,16 +624,16 @@ public class TestDefaultMemStore {
byte [] val = Bytes.toBytes("testval"); byte [] val = Bytes.toBytes("testval");
//Setting up memstore //Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
//Creating a snapshot //Creating a snapshot
memstore.snapshot(); memstore.snapshot();
assertEquals(3, memstore.getSnapshot().getCellsCount()); assertEquals(3, memstore.getSnapshot().getCellsCount());
//Adding value to "new" memstore //Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam ,qf4, val)); memstore.add(new KeyValue(row, fam ,qf4, val), null);
memstore.add(new KeyValue(row, fam ,qf5, val)); memstore.add(new KeyValue(row, fam ,qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
} }
@ -648,14 +653,14 @@ public class TestDefaultMemStore {
KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val); KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val);
long ts3 = ts2 + 1; long ts3 = ts2 + 1;
KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val); KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val);
memstore.add(put1); memstore.add(put1, null);
memstore.add(put2); memstore.add(put2, null);
memstore.add(put3); memstore.add(put3, null);
assertEquals(3, memstore.getActive().getCellsCount()); assertEquals(3, memstore.getActive().getCellsCount());
KeyValue del2 = new KeyValue(row, fam, qf1, ts2, KeyValue.Type.Delete, val); KeyValue del2 = new KeyValue(row, fam, qf1, ts2, KeyValue.Type.Delete, val);
memstore.delete(del2); memstore.add(del2, null);
List<Cell> expected = new ArrayList<Cell>(); List<Cell> expected = new ArrayList<Cell>();
expected.add(put3); expected.add(put3);
@ -683,15 +688,15 @@ public class TestDefaultMemStore {
KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val); KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val);
long ts3 = ts2 + 1; long ts3 = ts2 + 1;
KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val); KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val);
memstore.add(put1); memstore.add(put1, null);
memstore.add(put2); memstore.add(put2, null);
memstore.add(put3); memstore.add(put3, null);
assertEquals(3, memstore.getActive().getCellsCount()); assertEquals(3, memstore.getActive().getCellsCount());
KeyValue del2 = KeyValue del2 =
new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val); new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val);
memstore.delete(del2); memstore.add(del2, null);
List<Cell> expected = new ArrayList<Cell>(); List<Cell> expected = new ArrayList<Cell>();
expected.add(put3); expected.add(put3);
@ -721,14 +726,14 @@ public class TestDefaultMemStore {
KeyValue put3 = new KeyValue(row, fam, qf3, ts, val); KeyValue put3 = new KeyValue(row, fam, qf3, ts, val);
KeyValue put4 = new KeyValue(row, fam, qf3, ts+1, val); KeyValue put4 = new KeyValue(row, fam, qf3, ts+1, val);
memstore.add(put1); memstore.add(put1, null);
memstore.add(put2); memstore.add(put2, null);
memstore.add(put3); memstore.add(put3, null);
memstore.add(put4); memstore.add(put4, null);
KeyValue del = KeyValue del =
new KeyValue(row, fam, null, ts, KeyValue.Type.DeleteFamily, val); new KeyValue(row, fam, null, ts, KeyValue.Type.DeleteFamily, val);
memstore.delete(del); memstore.add(del, null);
List<Cell> expected = new ArrayList<Cell>(); List<Cell> expected = new ArrayList<Cell>();
expected.add(del); expected.add(del);
@ -751,9 +756,9 @@ public class TestDefaultMemStore {
byte [] qf = Bytes.toBytes("testqualifier"); byte [] qf = Bytes.toBytes("testqualifier");
byte [] val = Bytes.toBytes("testval"); byte [] val = Bytes.toBytes("testval");
long ts = System.nanoTime(); long ts = System.nanoTime();
memstore.add(new KeyValue(row, fam, qf, ts, val)); memstore.add(new KeyValue(row, fam, qf, ts, val), null);
KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val); KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val);
memstore.delete(delete); memstore.add(delete, null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
assertEquals(delete, memstore.getActive().first()); assertEquals(delete, memstore.getActive().first());
} }
@ -761,12 +766,12 @@ public class TestDefaultMemStore {
@Test @Test
public void testRetainsDeleteVersion() throws IOException { public void testRetainsDeleteVersion() throws IOException {
// add a put to memstore // add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"), null);
// now process a specific delete: // now process a specific delete:
KeyValue delete = KeyValueTestUtil.create( KeyValue delete = KeyValueTestUtil.create(
"row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care"); "row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care");
memstore.delete(delete); memstore.add(delete, null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
assertEquals(delete, memstore.getActive().first()); assertEquals(delete, memstore.getActive().first());
@ -775,12 +780,12 @@ public class TestDefaultMemStore {
@Test @Test
public void testRetainsDeleteColumn() throws IOException { public void testRetainsDeleteColumn() throws IOException {
// add a put to memstore // add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"), null);
// now process a specific delete: // now process a specific delete:
KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100, KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100,
KeyValue.Type.DeleteColumn, "dont-care"); KeyValue.Type.DeleteColumn, "dont-care");
memstore.delete(delete); memstore.add(delete, null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
assertEquals(delete, memstore.getActive().first()); assertEquals(delete, memstore.getActive().first());
@ -789,64 +794,17 @@ public class TestDefaultMemStore {
@Test @Test
public void testRetainsDeleteFamily() throws IOException { public void testRetainsDeleteFamily() throws IOException {
// add a put to memstore // add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care")); memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"), null);
// now process a specific delete: // now process a specific delete:
KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100, KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100,
KeyValue.Type.DeleteFamily, "dont-care"); KeyValue.Type.DeleteFamily, "dont-care");
memstore.delete(delete); memstore.add(delete, null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
assertEquals(delete, memstore.getActive().first()); assertEquals(delete, memstore.getActive().first());
} }
////////////////////////////////////
//Test for upsert with MSLAB
////////////////////////////////////
/**
* Test a pathological pattern that shows why we can't currently
* use the MSLAB for upsert workloads. This test inserts data
* in the following pattern:
*
* - row0001 through row1000 (fills up one 2M Chunk)
* - row0002 through row1001 (fills up another 2M chunk, leaves one reference
* to the first chunk
* - row0003 through row1002 (another chunk, another dangling reference)
*
* This causes OOME pretty quickly if we use MSLAB for upsert
* since each 2M chunk is held onto by a single reference.
*/
@Test
public void testUpsertMSLAB() throws Exception {
Configuration conf = HBaseConfiguration.create();
conf.setBoolean(SegmentFactory.USEMSLAB_KEY, true);
memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR);
int ROW_SIZE = 2048;
byte[] qualifier = new byte[ROW_SIZE - 4];
MemoryMXBean bean = ManagementFactory.getMemoryMXBean();
for (int i = 0; i < 3; i++) { System.gc(); }
long usageBefore = bean.getHeapMemoryUsage().getUsed();
long size = 0;
long ts=0;
for (int newValue = 0; newValue < 1000; newValue++) {
for (int row = newValue; row < newValue + 1000; row++) {
byte[] rowBytes = Bytes.toBytes(row);
size += memstore.updateColumnValue(rowBytes, FAMILY, qualifier, newValue, ++ts);
}
}
System.out.println("Wrote " + ts + " vals");
for (int i = 0; i < 3; i++) { System.gc(); }
long usageAfter = bean.getHeapMemoryUsage().getUsed();
System.out.println("Memory used: " + (usageAfter - usageBefore)
+ " (heapsize: " + memstore.heapSize() +
" size: " + size + ")");
}
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
// Helpers // Helpers
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
@ -864,7 +822,7 @@ public class TestDefaultMemStore {
public void testUpsertMemstoreSize() throws Exception { public void testUpsertMemstoreSize() throws Exception {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR); memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR);
long oldSize = memstore.size(); MemstoreSize oldSize = memstore.size();
List<Cell> l = new ArrayList<Cell>(); List<Cell> l = new ArrayList<Cell>();
KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
@ -874,16 +832,16 @@ public class TestDefaultMemStore {
kv1.setSequenceId(1); kv2.setSequenceId(1);kv3.setSequenceId(1); kv1.setSequenceId(1); kv2.setSequenceId(1);kv3.setSequenceId(1);
l.add(kv1); l.add(kv2); l.add(kv3); l.add(kv1); l.add(kv2); l.add(kv3);
this.memstore.upsert(l, 2);// readpoint is 2 this.memstore.upsert(l, 2, null);// readpoint is 2
long newSize = this.memstore.size(); MemstoreSize newSize = this.memstore.size();
assert(newSize > oldSize); assert (newSize.getDataSize() > oldSize.getDataSize());
//The kv1 should be removed. //The kv1 should be removed.
assert(memstore.getActive().getCellsCount() == 2); assert(memstore.getActive().getCellsCount() == 2);
KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v"); KeyValue kv4 = KeyValueTestUtil.create("r", "f", "q", 104, "v");
kv4.setSequenceId(1); kv4.setSequenceId(1);
l.clear(); l.add(kv4); l.clear(); l.add(kv4);
this.memstore.upsert(l, 3); this.memstore.upsert(l, 3, null);
assertEquals(newSize, this.memstore.size()); assertEquals(newSize, this.memstore.size());
//The kv2 should be removed. //The kv2 should be removed.
assert(memstore.getActive().getCellsCount() == 2); assert(memstore.getActive().getCellsCount() == 2);
@ -910,7 +868,7 @@ public class TestDefaultMemStore {
assertEquals(t, Long.MAX_VALUE); assertEquals(t, Long.MAX_VALUE);
// test the case that the timeOfOldestEdit is updated after a KV add // test the case that the timeOfOldestEdit is updated after a KV add
memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, "v")); memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, "v"), null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
// snapshot() will reset timeOfOldestEdit. The method will also assert the // snapshot() will reset timeOfOldestEdit. The method will also assert the
@ -918,7 +876,7 @@ public class TestDefaultMemStore {
t = runSnapshot(memstore); t = runSnapshot(memstore);
// test the case that the timeOfOldestEdit is updated after a KV delete // test the case that the timeOfOldestEdit is updated after a KV delete
memstore.delete(KeyValueTestUtil.create("r", "f", "q", 100, "v")); memstore.add(KeyValueTestUtil.create("r", "f", "q", 100, KeyValue.Type.Delete, "v"), null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
t = runSnapshot(memstore); t = runSnapshot(memstore);
@ -928,7 +886,7 @@ public class TestDefaultMemStore {
KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v");
kv1.setSequenceId(100); kv1.setSequenceId(100);
l.add(kv1); l.add(kv1);
memstore.upsert(l, 1000); memstore.upsert(l, 1000, null);
t = memstore.timeOfOldestEdit(); t = memstore.timeOfOldestEdit();
assertTrue(t == 1234); assertTrue(t == 1234);
} finally { } finally {
@ -1041,7 +999,7 @@ public class TestDefaultMemStore {
for (int ii = 0; ii < QUALIFIER_COUNT; ii++) { for (int ii = 0; ii < QUALIFIER_COUNT; ii++) {
byte [] row = Bytes.toBytes(i); byte [] row = Bytes.toBytes(i);
byte [] qf = makeQualifier(i, ii); byte [] qf = makeQualifier(i, ii);
hmc.add(new KeyValue(row, FAMILY, qf, timestamp, qf)); hmc.add(new KeyValue(row, FAMILY, qf, timestamp, qf), null);
} }
} }
return ROW_COUNT; return ROW_COUNT;
@ -1088,7 +1046,7 @@ public class TestDefaultMemStore {
for (int ii = 0; ii < QUALIFIER_COUNT ; ii++) { for (int ii = 0; ii < QUALIFIER_COUNT ; ii++) {
byte [] row = Bytes.toBytes(i); byte [] row = Bytes.toBytes(i);
byte [] qf = makeQualifier(i, ii); byte [] qf = makeQualifier(i, ii);
mem.add(new KeyValue(row, FAMILY, qf, timestamp, qf)); mem.add(new KeyValue(row, FAMILY, qf, timestamp, qf), null);
} }
} }
} }

View File

@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -179,7 +180,7 @@ public class TestHMobStore {
KeyValue[] keys = new KeyValue[] { key1, key2, key3 }; KeyValue[] keys = new KeyValue[] { key1, key2, key3 };
int maxKeyCount = keys.length; int maxKeyCount = keys.length;
StoreFileWriter mobWriter = store.createWriterInTmp(currentDate, maxKeyCount, StoreFileWriter mobWriter = store.createWriterInTmp(currentDate, maxKeyCount,
hcd.getCompactionCompression(), region.getRegionInfo().getStartKey()); hcd.getCompactionCompressionType(), region.getRegionInfo().getStartKey());
mobFilePath = mobWriter.getPath(); mobFilePath = mobWriter.getPath();
mobWriter.append(key1); mobWriter.append(key1);
@ -209,12 +210,12 @@ public class TestHMobStore {
init(name.getMethodName(), conf, false); init(name.getMethodName(), conf, false);
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
Scan scan = new Scan(get); Scan scan = new Scan(get);
InternalScanner scanner = (InternalScanner) store.getScanner(scan, InternalScanner scanner = (InternalScanner) store.getScanner(scan,
@ -223,7 +224,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
//Compare //Compare
@ -244,20 +245,20 @@ public class TestHMobStore {
init(name.getMethodName(), conf, false); init(name.getMethodName(), conf, false);
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
//flush //flush
flush(3); flush(3);
@ -268,7 +269,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
//Compare //Compare
@ -288,20 +289,20 @@ public class TestHMobStore {
init(name.getMethodName(), conf, false); init(name.getMethodName(), conf, false);
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
//flush //flush
flush(3); flush(3);
@ -313,7 +314,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
//Compare //Compare
@ -336,20 +337,20 @@ public class TestHMobStore {
init(name.getMethodName(), conf, false); init(name.getMethodName(), conf, false);
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
Scan scan = new Scan(get); Scan scan = new Scan(get);
InternalScanner scanner = (InternalScanner) store.getScanner(scan, InternalScanner scanner = (InternalScanner) store.getScanner(scan,
@ -358,7 +359,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
//Compare //Compare
@ -385,20 +386,20 @@ public class TestHMobStore {
init(name.getMethodName(), conf, hcd, false); init(name.getMethodName(), conf, hcd, false);
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
//flush //flush
flush(3); flush(3);
@ -410,7 +411,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
//Compare //Compare
@ -505,14 +506,14 @@ public class TestHMobStore {
init(name.getMethodName(), conf, hcd, false); init(name.getMethodName(), conf, hcd, false);
this.store.add(new KeyValue(row, family, qf1, 1, value)); this.store.add(new KeyValue(row, family, qf1, 1, value), null);
this.store.add(new KeyValue(row, family, qf2, 1, value)); this.store.add(new KeyValue(row, family, qf2, 1, value), null);
this.store.add(new KeyValue(row, family, qf3, 1, value)); this.store.add(new KeyValue(row, family, qf3, 1, value), null);
flush(1); flush(1);
this.store.add(new KeyValue(row, family, qf4, 1, value)); this.store.add(new KeyValue(row, family, qf4, 1, value), null);
this.store.add(new KeyValue(row, family, qf5, 1, value)); this.store.add(new KeyValue(row, family, qf5, 1, value), null);
this.store.add(new KeyValue(row, family, qf6, 1, value)); this.store.add(new KeyValue(row, family, qf6, 1, value), null);
flush(2); flush(2);
Collection<StoreFile> storefiles = this.store.getStorefiles(); Collection<StoreFile> storefiles = this.store.getStorefiles();
@ -526,7 +527,7 @@ public class TestHMobStore {
List<Cell> results = new ArrayList<Cell>(); List<Cell> results = new ArrayList<Cell>();
scanner.next(results); scanner.next(results);
Collections.sort(results, KeyValue.COMPARATOR); Collections.sort(results, CellComparator.COMPARATOR);
scanner.close(); scanner.close();
Assert.assertEquals(expected.size(), results.size()); Assert.assertEquals(expected.size(), results.size());
for(int i=0; i<results.size(); i++) { for(int i=0; i<results.size(); i++) {

View File

@ -150,7 +150,6 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -356,7 +355,7 @@ public class TestHRegion {
} finally { } finally {
assertTrue("The regionserver should have thrown an exception", threwIOE); assertTrue("The regionserver should have thrown an exception", threwIOE);
} }
long sz = store.getFlushableSize(); long sz = store.getSizeToFlush().getDataSize();
assertTrue("flushable size should be zero, but it is " + sz, sz == 0); assertTrue("flushable size should be zero, but it is " + sz, sz == 0);
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }
@ -400,7 +399,7 @@ public class TestHRegion {
assertTrue(onePutSize > 0); assertTrue(onePutSize > 0);
region.flush(true); region.flush(true);
assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
assertEquals("flushable size should be zero", 0, store.getFlushableSize()); assertEquals("flushable size should be zero", 0, store.getSizeToFlush().getDataSize());
// save normalCPHost and replaced by mockedCPHost, which will cancel flush requests // save normalCPHost and replaced by mockedCPHost, which will cancel flush requests
RegionCoprocessorHost normalCPHost = region.getCoprocessorHost(); RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
@ -411,13 +410,14 @@ public class TestHRegion {
region.put(put); region.put(put);
region.flush(true); region.flush(true);
assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize()); assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize());
assertEquals("flushable size should NOT be zero", onePutSize, store.getFlushableSize()); assertEquals("flushable size should NOT be zero", onePutSize,
store.getSizeToFlush().getDataSize());
// set normalCPHost and flush again, the snapshot will be flushed // set normalCPHost and flush again, the snapshot will be flushed
region.setCoprocessorHost(normalCPHost); region.setCoprocessorHost(normalCPHost);
region.flush(true); region.flush(true);
assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
assertEquals("flushable size should be zero", 0, store.getFlushableSize()); assertEquals("flushable size should be zero", 0, store.getSizeToFlush().getDataSize());
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }
@ -452,9 +452,10 @@ public class TestHRegion {
fail("Should have failed with IOException"); fail("Should have failed with IOException");
} catch (IOException expected) { } catch (IOException expected) {
} }
long expectedSize = onePutSize * 2 - ClassSize.ARRAY; long expectedSize = onePutSize * 2;
assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize()); assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize());
assertEquals("flushable size should be incremented", expectedSize, store.getFlushableSize()); assertEquals("flushable size should be incremented", expectedSize,
store.getSizeToFlush().getDataSize());
region.setCoprocessorHost(null); region.setCoprocessorHost(null);
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
@ -524,14 +525,14 @@ public class TestHRegion {
p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null)); p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null)); p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
region.put(p2); region.put(p2);
long expectedSize = sizeOfOnePut * 3- ClassSize.ARRAY; long expectedSize = sizeOfOnePut * 3;
Assert.assertEquals(expectedSize, region.getMemstoreSize()); Assert.assertEquals(expectedSize, region.getMemstoreSize());
// Do a successful flush. It will clear the snapshot only. Thats how flushes work. // Do a successful flush. It will clear the snapshot only. Thats how flushes work.
// If already a snapshot, we clear it else we move the memstore to be snapshot and flush // If already a snapshot, we clear it else we move the memstore to be snapshot and flush
// it // it
region.flush(true); region.flush(true);
// Make sure our memory accounting is right. // Make sure our memory accounting is right.
Assert.assertEquals(sizeOfOnePut * 2 - ClassSize.ARRAY, region.getMemstoreSize()); Assert.assertEquals(sizeOfOnePut * 2, region.getMemstoreSize());
} finally { } finally {
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }

View File

@ -955,8 +955,8 @@ public class TestHRegionReplayEvents {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long newSnapshotSize = store.getSnapshotSize(); MemstoreSize newSnapshotSize = store.getSizeOfSnapshot();
assertTrue(newSnapshotSize == 0); assertTrue(newSnapshotSize.getDataSize() == 0);
// assert that the region memstore is empty // assert that the region memstore is empty
long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); long newRegionMemstoreSize = secondaryRegion.getMemstoreSize();

View File

@ -117,9 +117,9 @@ public class TestMemStoreChunkPool {
DefaultMemStore memstore = new DefaultMemStore(); DefaultMemStore memstore = new DefaultMemStore();
// Setting up memstore // Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
// Creating a snapshot // Creating a snapshot
MemStoreSnapshot snapshot = memstore.snapshot(); MemStoreSnapshot snapshot = memstore.snapshot();
@ -127,8 +127,8 @@ public class TestMemStoreChunkPool {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf4, val)); memstore.add(new KeyValue(row, fam, qf4, val), null);
memstore.add(new KeyValue(row, fam, qf5, val)); memstore.add(new KeyValue(row, fam, qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
memstore.clearSnapshot(snapshot.getId()); memstore.clearSnapshot(snapshot.getId());
@ -154,9 +154,9 @@ public class TestMemStoreChunkPool {
DefaultMemStore memstore = new DefaultMemStore(); DefaultMemStore memstore = new DefaultMemStore();
// Setting up memstore // Setting up memstore
memstore.add(new KeyValue(row, fam, qf1, val)); memstore.add(new KeyValue(row, fam, qf1, val), null);
memstore.add(new KeyValue(row, fam, qf2, val)); memstore.add(new KeyValue(row, fam, qf2, val), null);
memstore.add(new KeyValue(row, fam, qf3, val)); memstore.add(new KeyValue(row, fam, qf3, val), null);
// Creating a snapshot // Creating a snapshot
MemStoreSnapshot snapshot = memstore.snapshot(); MemStoreSnapshot snapshot = memstore.snapshot();
@ -164,8 +164,8 @@ public class TestMemStoreChunkPool {
// Adding value to "new" memstore // Adding value to "new" memstore
assertEquals(0, memstore.getActive().getCellsCount()); assertEquals(0, memstore.getActive().getCellsCount());
memstore.add(new KeyValue(row, fam, qf4, val)); memstore.add(new KeyValue(row, fam, qf4, val), null);
memstore.add(new KeyValue(row, fam, qf5, val)); memstore.add(new KeyValue(row, fam, qf5, val), null);
assertEquals(2, memstore.getActive().getCellsCount()); assertEquals(2, memstore.getActive().getCellsCount());
// opening scanner before clear the snapshot // opening scanner before clear the snapshot
@ -188,8 +188,8 @@ public class TestMemStoreChunkPool {
// Creating another snapshot // Creating another snapshot
snapshot = memstore.snapshot(); snapshot = memstore.snapshot();
// Adding more value // Adding more value
memstore.add(new KeyValue(row, fam, qf6, val)); memstore.add(new KeyValue(row, fam, qf6, val), null);
memstore.add(new KeyValue(row, fam, qf7, val)); memstore.add(new KeyValue(row, fam, qf7, val), null);
// opening scanners // opening scanners
scanners = memstore.getScanners(0); scanners = memstore.getScanners(0);
// close scanners before clear the snapshot // close scanners before clear the snapshot

View File

@ -128,7 +128,7 @@ public class TestPerColumnFamilyFlush {
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 200 * 1024);
conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName());
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN,
100 * 1024); 40 * 1024);
// Intialize the region // Intialize the region
Region region = initHRegion("testSelectiveFlushWithDataCompaction", conf); Region region = initHRegion("testSelectiveFlushWithDataCompaction", conf);
// Add 1200 entries for CF1, 100 for CF2 and 50 for CF3 // Add 1200 entries for CF1, 100 for CF2 and 50 for CF3
@ -151,9 +151,9 @@ public class TestPerColumnFamilyFlush {
long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstore = getWAL(region) long smallestSeqInRegionCurrentMemstore = getWAL(region)
@ -166,34 +166,33 @@ public class TestPerColumnFamilyFlush {
// Some other sanity checks. // Some other sanity checks.
assertTrue(smallestSeqCF1 < smallestSeqCF2); assertTrue(smallestSeqCF1 < smallestSeqCF2);
assertTrue(smallestSeqCF2 < smallestSeqCF3); assertTrue(smallestSeqCF2 < smallestSeqCF3);
assertTrue(cf1MemstoreSize > 0); assertTrue(cf1MemstoreSize.getDataSize() > 0);
assertTrue(cf2MemstoreSize > 0); assertTrue(cf2MemstoreSize.getDataSize() > 0);
assertTrue(cf3MemstoreSize > 0); assertTrue(cf3MemstoreSize.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
assertEquals( assertEquals(totalMemstoreSize, cf1MemstoreSize.getDataSize() + cf2MemstoreSize.getDataSize()
totalMemstoreSize + (3 * (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD)), + cf3MemstoreSize.getDataSize());
cf1MemstoreSize + cf2MemstoreSize + cf3MemstoreSize);
// Flush! // Flush!
region.flush(false); region.flush(false);
// Will use these to check if anything changed. // Will use these to check if anything changed.
long oldCF2MemstoreSize = cf2MemstoreSize; MemstoreSize oldCF2MemstoreSize = cf2MemstoreSize;
long oldCF3MemstoreSize = cf3MemstoreSize; MemstoreSize oldCF3MemstoreSize = cf3MemstoreSize;
// Recalculate everything // Recalculate everything
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
smallestSeqInRegionCurrentMemstore = getWAL(region) smallestSeqInRegionCurrentMemstore = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
// We should have cleared out only CF1, since we chose the flush thresholds // We should have cleared out only CF1, since we chose the flush thresholds
// and number of puts accordingly. // and number of puts accordingly.
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf1MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf1MemstoreSize);
// Nothing should have happened to CF2, ... // Nothing should have happened to CF2, ...
assertEquals(cf2MemstoreSize, oldCF2MemstoreSize); assertEquals(cf2MemstoreSize, oldCF2MemstoreSize);
// ... or CF3 // ... or CF3
@ -202,9 +201,7 @@ public class TestPerColumnFamilyFlush {
// LSN in the memstore of CF2. // LSN in the memstore of CF2.
assertEquals(smallestSeqInRegionCurrentMemstore, smallestSeqCF2); assertEquals(smallestSeqInRegionCurrentMemstore, smallestSeqCF2);
// Of course, this should hold too. // Of course, this should hold too.
assertEquals( assertEquals(totalMemstoreSize, cf2MemstoreSize.getDataSize() + cf3MemstoreSize.getDataSize());
totalMemstoreSize + (2 * (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD)),
cf2MemstoreSize + cf3MemstoreSize);
// Now add more puts (mostly for CF2), so that we only flush CF2 this time. // Now add more puts (mostly for CF2), so that we only flush CF2 this time.
for (int i = 1200; i < 2400; i++) { for (int i = 1200; i < 2400; i++) {
@ -217,26 +214,25 @@ public class TestPerColumnFamilyFlush {
} }
// How much does the CF3 memstore occupy? Will be used later. // How much does the CF3 memstore occupy? Will be used later.
oldCF3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); oldCF3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
// Flush again // Flush again
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
smallestSeqInRegionCurrentMemstore = getWAL(region) smallestSeqInRegionCurrentMemstore = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
// CF1 and CF2, both should be absent. // CF1 and CF2, both should be absent.
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf1MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf1MemstoreSize);
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf2MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSize);
// CF3 shouldn't have been touched. // CF3 shouldn't have been touched.
assertEquals(cf3MemstoreSize, oldCF3MemstoreSize); assertEquals(cf3MemstoreSize, oldCF3MemstoreSize);
assertEquals(totalMemstoreSize + (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD), assertEquals(totalMemstoreSize, cf3MemstoreSize.getDataSize());
cf3MemstoreSize);
assertEquals(smallestSeqInRegionCurrentMemstore, smallestSeqCF3); assertEquals(smallestSeqInRegionCurrentMemstore, smallestSeqCF3);
// What happens when we hit the memstore limit, but we are not able to find // What happens when we hit the memstore limit, but we are not able to find
@ -288,35 +284,34 @@ public class TestPerColumnFamilyFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSize > 0); assertTrue(cf1MemstoreSize.getDataSize() > 0);
assertTrue(cf2MemstoreSize > 0); assertTrue(cf2MemstoreSize.getDataSize() > 0);
assertTrue(cf3MemstoreSize > 0); assertTrue(cf3MemstoreSize.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
assertEquals( assertEquals(totalMemstoreSize, cf1MemstoreSize.getDataSize() + cf2MemstoreSize.getDataSize()
totalMemstoreSize + (3 * (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD)), + cf3MemstoreSize.getDataSize());
cf1MemstoreSize + cf2MemstoreSize + cf3MemstoreSize);
// Flush! // Flush!
region.flush(false); region.flush(false);
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
long smallestSeqInRegionCurrentMemstore = long smallestSeqInRegionCurrentMemstore =
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
// Everything should have been cleared // Everything should have been cleared
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf1MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf1MemstoreSize);
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf2MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSize);
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf3MemstoreSize); assertEquals(MemstoreSize.EMPTY_SIZE, cf3MemstoreSize);
assertEquals(0, totalMemstoreSize); assertEquals(0, totalMemstoreSize);
assertEquals(HConstants.NO_SEQNUM, smallestSeqInRegionCurrentMemstore); assertEquals(HConstants.NO_SEQNUM, smallestSeqInRegionCurrentMemstore);
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
@ -337,10 +332,10 @@ public class TestPerColumnFamilyFlush {
private void doTestLogReplay() throws Exception { private void doTestLogReplay() throws Exception {
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 20000); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 10000);
// Carefully chosen limits so that the memstore just flushes when we're done // Carefully chosen limits so that the memstore just flushes when we're done
conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName()); conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName());
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 10000); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 2500);
final int numRegionServers = 4; final int numRegionServers = 4;
try { try {
TEST_UTIL.startMiniCluster(numRegionServers); TEST_UTIL.startMiniCluster(numRegionServers);
@ -378,18 +373,16 @@ public class TestPerColumnFamilyFlush {
totalMemstoreSize = desiredRegion.getMemstoreSize(); totalMemstoreSize = desiredRegion.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize(); cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getSizeOfMemStore().getDataSize();
cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getMemStoreSize(); cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getSizeOfMemStore().getDataSize();
cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getMemStoreSize(); cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getSizeOfMemStore().getDataSize();
// CF1 Should have been flushed // CF1 Should have been flushed
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, cf1MemstoreSize); assertEquals(0, cf1MemstoreSize);
// CF2 and CF3 shouldn't have been flushed. // CF2 and CF3 shouldn't have been flushed.
assertTrue(cf2MemstoreSize > 0); assertTrue(cf2MemstoreSize > 0);
assertTrue(cf3MemstoreSize > 0); assertTrue(cf3MemstoreSize > 0);
assertEquals( assertEquals(totalMemstoreSize, cf2MemstoreSize + cf3MemstoreSize);
totalMemstoreSize + (2 * (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD)),
cf2MemstoreSize + cf3MemstoreSize);
// Wait for the RS report to go across to the master, so that the master // Wait for the RS report to go across to the master, so that the master
// is aware of which sequence ids have been flushed, before we kill the RS. // is aware of which sequence ids have been flushed, before we kill the RS.
@ -526,12 +519,9 @@ public class TestPerColumnFamilyFlush {
}); });
LOG.info("Finished waiting on flush after too many WALs..."); LOG.info("Finished waiting on flush after too many WALs...");
// Individual families should have been flushed. // Individual families should have been flushed.
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(0, desiredRegion.getStore(FAMILY1).getMemStoreSize());
desiredRegion.getStore(FAMILY1).getMemStoreSize()); assertEquals(0, desiredRegion.getStore(FAMILY2).getMemStoreSize());
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(0, desiredRegion.getStore(FAMILY3).getMemStoreSize());
desiredRegion.getStore(FAMILY2).getMemStoreSize());
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD,
desiredRegion.getStore(FAMILY3).getMemStoreSize());
// let WAL cleanOldLogs // let WAL cleanOldLogs
assertNull(getWAL(desiredRegion).rollWriter(true)); assertNull(getWAL(desiredRegion).rollWriter(true));
assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs); assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs);

View File

@ -24,13 +24,10 @@ import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.*; import static org.mockito.Mockito.*;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.CoordinatedStateManagerFactory; import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -42,7 +39,6 @@ import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -177,6 +173,7 @@ public class TestRegionMergeTransaction {
HStore storeMock = Mockito.mock(HStore.class); HStore storeMock = Mockito.mock(HStore.class);
when(storeMock.hasReferences()).thenReturn(true); when(storeMock.hasReferences()).thenReturn(true);
when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf")); when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
when(storeMock.getSizeToFlush()).thenReturn(new MemstoreSize());
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of()); when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
this.region_a.stores.put(Bytes.toBytes(""), storeMock); this.region_a.stores.put(Bytes.toBytes(""), storeMock);
RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a, RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(this.region_a,

View File

@ -638,7 +638,7 @@ public class TestReversibleScanners {
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
for (int j = 0; j < QUALSIZE; j++) { for (int j = 0; j < QUALSIZE; j++) {
if (i % 2 == 0) { if (i % 2 == 0) {
memstore.add(makeKV(i, j)); memstore.add(makeKV(i, j), null);
} else { } else {
writers[(i + j) % writers.length].append(makeKV(i, j)); writers[(i + j) % writers.length].append(makeKV(i, j));
} }
@ -669,7 +669,7 @@ public class TestReversibleScanners {
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
for (int j = 0; j < QUALSIZE; j++) { for (int j = 0; j < QUALSIZE; j++) {
if ((i + j) % 2 == 0) { if ((i + j) % 2 == 0) {
memstore.add(makeKV(i, j)); memstore.add(makeKV(i, j), null);
} }
} }
} }
@ -678,7 +678,7 @@ public class TestReversibleScanners {
for (int i = 0; i < ROWSIZE; i++) { for (int i = 0; i < ROWSIZE; i++) {
for (int j = 0; j < QUALSIZE; j++) { for (int j = 0; j < QUALSIZE; j++) {
if ((i + j) % 2 == 1) { if ((i + j) % 2 == 1) {
memstore.add(makeKV(i, j)); memstore.add(makeKV(i, j), null);
} }
} }
} }

View File

@ -28,13 +28,10 @@ import static org.mockito.Mockito.*;
import org.mockito.Mockito; import org.mockito.Mockito;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
@ -42,7 +39,6 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@ -176,6 +172,7 @@ public class TestSplitTransaction {
HStore storeMock = Mockito.mock(HStore.class); HStore storeMock = Mockito.mock(HStore.class);
when(storeMock.hasReferences()).thenReturn(true); when(storeMock.hasReferences()).thenReturn(true);
when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf")); when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
when(storeMock.getSizeToFlush()).thenReturn(new MemstoreSize());
when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of()); when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
this.parent.stores.put(Bytes.toBytes(""), storeMock); this.parent.stores.put(Bytes.toBytes(""), storeMock);

View File

@ -53,14 +53,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
@ -81,7 +79,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
@ -220,10 +217,11 @@ public class TestStore {
// Initialize region // Initialize region
init(name.getMethodName(), conf); init(name.getMethodName(), conf);
long size = store.memstore.getFlushableSize(); MemstoreSize size = store.memstore.getFlushableSize();
Assert.assertEquals(0, size); Assert.assertEquals(0, size.getDataSize());
LOG.info("Adding some data"); LOG.info("Adding some data");
long kvSize = store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); MemstoreSize kvSize = new MemstoreSize();
store.add(new KeyValue(row, family, qf1, 1, (byte[])null), kvSize);
size = store.memstore.getFlushableSize(); size = store.memstore.getFlushableSize();
Assert.assertEquals(kvSize, size); Assert.assertEquals(kvSize, size);
// Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right. // Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right.
@ -236,7 +234,8 @@ public class TestStore {
} }
size = store.memstore.getFlushableSize(); size = store.memstore.getFlushableSize();
Assert.assertEquals(kvSize, size); Assert.assertEquals(kvSize, size);
store.add(new KeyValue(row, family, qf2, 2, (byte[])null)); MemstoreSize kvSize2 = new MemstoreSize();
store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2);
// Even though we add a new kv, we expect the flushable size to be 'same' since we have // Even though we add a new kv, we expect the flushable size to be 'same' since we have
// not yet cleared the snapshot -- the above flush failed. // not yet cleared the snapshot -- the above flush failed.
Assert.assertEquals(kvSize, size); Assert.assertEquals(kvSize, size);
@ -244,10 +243,10 @@ public class TestStore {
flushStore(store, id++); flushStore(store, id++);
size = store.memstore.getFlushableSize(); size = store.memstore.getFlushableSize();
// Size should be the foreground kv size. // Size should be the foreground kv size.
Assert.assertEquals(kvSize, size); Assert.assertEquals(kvSize2, size);
flushStore(store, id++); flushStore(store, id++);
size = store.memstore.getFlushableSize(); size = store.memstore.getFlushableSize();
Assert.assertEquals(0, size); Assert.assertEquals(MemstoreSize.EMPTY_SIZE, size);
return null; return null;
} }
}); });
@ -317,9 +316,9 @@ public class TestStore {
for (int i = 1; i <= storeFileNum; i++) { for (int i = 1; i <= storeFileNum; i++) {
LOG.info("Adding some data for the store file #" + i); LOG.info("Adding some data for the store file #" + i);
timeStamp = EnvironmentEdgeManager.currentTime(); timeStamp = EnvironmentEdgeManager.currentTime();
this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null)); this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);
this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null)); this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);
this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null)); this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);
flush(i); flush(i);
edge.incrementTime(sleepTime); edge.incrementTime(sleepTime);
} }
@ -371,9 +370,9 @@ public class TestStore {
int storeFileNum = 4; int storeFileNum = 4;
for (int i = 1; i <= storeFileNum; i++) { for (int i = 1; i <= storeFileNum; i++) {
LOG.info("Adding some data for the store file #"+i); LOG.info("Adding some data for the store file #"+i);
this.store.add(new KeyValue(row, family, qf1, i, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, i, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf2, i, (byte[])null)); this.store.add(new KeyValue(row, family, qf2, i, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf3, i, (byte[])null)); this.store.add(new KeyValue(row, family, qf3, i, (byte[])null), null);
flush(i); flush(i);
} }
// after flush; check the lowest time stamp // after flush; check the lowest time stamp
@ -424,8 +423,8 @@ public class TestStore {
public void testEmptyStoreFile() throws IOException { public void testEmptyStoreFile() throws IOException {
init(this.name.getMethodName()); init(this.name.getMethodName());
// Write a store file. // Write a store file.
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
flush(1); flush(1);
// Now put in place an empty store file. Its a little tricky. Have to // Now put in place an empty store file. Its a little tricky. Have to
// do manually with hacked in sequence id. // do manually with hacked in sequence id.
@ -462,12 +461,12 @@ public class TestStore {
init(this.name.getMethodName()); init(this.name.getMethodName());
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
//Get //Get
result = HBaseTestingUtility.getFromStoreFile(store, result = HBaseTestingUtility.getFromStoreFile(store,
@ -486,20 +485,20 @@ public class TestStore {
init(this.name.getMethodName()); init(this.name.getMethodName());
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
//flush //flush
flush(3); flush(3);
@ -525,20 +524,20 @@ public class TestStore {
init(this.name.getMethodName()); init(this.name.getMethodName());
//Put data in memstore //Put data in memstore
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
//flush //flush
flush(1); flush(1);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null), null);
//flush //flush
flush(2); flush(2);
//Add more data //Add more data
this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null), null);
this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null), null);
//Get //Get
result = HBaseTestingUtility.getFromStoreFile(store, result = HBaseTestingUtility.getFromStoreFile(store,
@ -565,186 +564,11 @@ public class TestStore {
} }
} }
//////////////////////////////////////////////////////////////////////////////
// IncrementColumnValue tests
//////////////////////////////////////////////////////////////////////////////
/*
* test the internal details of how ICV works, especially during a flush scenario.
*/
@Test
public void testIncrementColumnValue_ICVDuringFlush()
throws IOException, InterruptedException {
init(this.name.getMethodName());
long oldValue = 1L;
long newValue = 3L;
this.store.add(new KeyValue(row, family, qf1,
System.currentTimeMillis(),
Bytes.toBytes(oldValue)));
// snapshot the store.
this.store.snapshot();
// add other things:
this.store.add(new KeyValue(row, family, qf2,
System.currentTimeMillis(),
Bytes.toBytes(oldValue)));
// update during the snapshot.
long ret = this.store.updateColumnValue(row, family, qf1, newValue);
// memstore should have grown by some amount.
Assert.assertTrue(ret > 0);
// then flush.
flushStore(store, id++);
Assert.assertEquals(1, this.store.getStorefiles().size());
// from the one we inserted up there, and a new one
Assert.assertEquals(2, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
// how many key/values for this row are there?
Get get = new Get(row);
get.addColumn(family, qf1);
get.setMaxVersions(); // all versions.
List<Cell> results = new ArrayList<Cell>();
results = HBaseTestingUtility.getFromStoreFile(store, get);
Assert.assertEquals(2, results.size());
long ts1 = results.get(0).getTimestamp();
long ts2 = results.get(1).getTimestamp();
Assert.assertTrue(ts1 > ts2);
Assert.assertEquals(newValue, Bytes.toLong(CellUtil.cloneValue(results.get(0))));
Assert.assertEquals(oldValue, Bytes.toLong(CellUtil.cloneValue(results.get(1))));
}
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
EnvironmentEdgeManagerTestHelper.reset(); EnvironmentEdgeManagerTestHelper.reset();
} }
@Test
public void testICV_negMemstoreSize() throws IOException {
init(this.name.getMethodName());
long time = 100;
ManualEnvironmentEdge ee = new ManualEnvironmentEdge();
ee.setValue(time);
EnvironmentEdgeManagerTestHelper.injectEdge(ee);
long newValue = 3L;
long size = 0;
size += this.store.add(new KeyValue(Bytes.toBytes("200909091000"), family, qf1,
System.currentTimeMillis(), Bytes.toBytes(newValue)));
size += this.store.add(new KeyValue(Bytes.toBytes("200909091200"), family, qf1,
System.currentTimeMillis(), Bytes.toBytes(newValue)));
size += this.store.add(new KeyValue(Bytes.toBytes("200909091300"), family, qf1,
System.currentTimeMillis(), Bytes.toBytes(newValue)));
size += this.store.add(new KeyValue(Bytes.toBytes("200909091400"), family, qf1,
System.currentTimeMillis(), Bytes.toBytes(newValue)));
size += this.store.add(new KeyValue(Bytes.toBytes("200909091500"), family, qf1,
System.currentTimeMillis(), Bytes.toBytes(newValue)));
for ( int i = 0 ; i < 10000 ; ++i) {
newValue++;
long ret = this.store.updateColumnValue(row, family, qf1, newValue);
long ret2 = this.store.updateColumnValue(row2, family, qf1, newValue);
if (ret != 0) System.out.println("ret: " + ret);
if (ret2 != 0) System.out.println("ret2: " + ret2);
Assert.assertTrue("ret: " + ret, ret >= 0);
size += ret;
Assert.assertTrue("ret2: " + ret2, ret2 >= 0);
size += ret2;
if (i % 1000 == 0)
ee.setValue(++time);
}
long computedSize=0;
for (Cell cell : ((AbstractMemStore)this.store.memstore).getActive().getCellSet()) {
long kvsize = DefaultMemStore.heapSizeChange(cell, true);
//System.out.println(kv + " size= " + kvsize + " kvsize= " + kv.heapSize());
computedSize += kvsize;
}
Assert.assertEquals(computedSize, size);
}
@Test
public void testIncrementColumnValue_SnapshotFlushCombo() throws Exception {
ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
EnvironmentEdgeManagerTestHelper.injectEdge(mee);
init(this.name.getMethodName());
long oldValue = 1L;
long newValue = 3L;
this.store.add(new KeyValue(row, family, qf1,
EnvironmentEdgeManager.currentTime(),
Bytes.toBytes(oldValue)));
// snapshot the store.
this.store.snapshot();
// update during the snapshot, the exact same TS as the Put (lololol)
long ret = this.store.updateColumnValue(row, family, qf1, newValue);
// memstore should have grown by some amount.
Assert.assertTrue(ret > 0);
// then flush.
flushStore(store, id++);
Assert.assertEquals(1, this.store.getStorefiles().size());
Assert.assertEquals(1, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount());
// now increment again:
newValue += 1;
this.store.updateColumnValue(row, family, qf1, newValue);
// at this point we have a TS=1 in snapshot, and a TS=2 in kvset, so increment again:
newValue += 1;
this.store.updateColumnValue(row, family, qf1, newValue);
// the second TS should be TS=2 or higher., even though 'time=1' right now.
// how many key/values for this row are there?
Get get = new Get(row);
get.addColumn(family, qf1);
get.setMaxVersions(); // all versions.
List<Cell> results = new ArrayList<Cell>();
results = HBaseTestingUtility.getFromStoreFile(store, get);
Assert.assertEquals(2, results.size());
long ts1 = results.get(0).getTimestamp();
long ts2 = results.get(1).getTimestamp();
Assert.assertTrue(ts1 > ts2);
Assert.assertEquals(newValue, Bytes.toLong(CellUtil.cloneValue(results.get(0))));
Assert.assertEquals(oldValue, Bytes.toLong(CellUtil.cloneValue(results.get(1))));
mee.setValue(2); // time goes up slightly
newValue += 1;
this.store.updateColumnValue(row, family, qf1, newValue);
results = HBaseTestingUtility.getFromStoreFile(store, get);
Assert.assertEquals(2, results.size());
ts1 = results.get(0).getTimestamp();
ts2 = results.get(1).getTimestamp();
Assert.assertTrue(ts1 > ts2);
Assert.assertEquals(newValue, Bytes.toLong(CellUtil.cloneValue(results.get(0))));
Assert.assertEquals(oldValue, Bytes.toLong(CellUtil.cloneValue(results.get(1))));
}
@Test @Test
public void testHandleErrorsInFlush() throws Exception { public void testHandleErrorsInFlush() throws Exception {
LOG.info("Setting up a faulty file system that cannot write"); LOG.info("Setting up a faulty file system that cannot write");
@ -766,9 +590,9 @@ public class TestStore {
init(name.getMethodName(), conf); init(name.getMethodName(), conf);
LOG.info("Adding some data"); LOG.info("Adding some data");
store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
store.add(new KeyValue(row, family, qf2, 1, (byte[])null)); store.add(new KeyValue(row, family, qf2, 1, (byte[])null), null);
store.add(new KeyValue(row, family, qf3, 1, (byte[])null)); store.add(new KeyValue(row, family, qf3, 1, (byte[])null), null);
LOG.info("Before flush, we should have no files"); LOG.info("Before flush, we should have no files");
@ -899,7 +723,7 @@ public class TestStore {
List<Cell> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family); List<Cell> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family);
for (Cell kv : kvList1) { for (Cell kv : kvList1) {
this.store.add(KeyValueUtil.ensureKeyValue(kv)); this.store.add(kv, null);
} }
this.store.snapshot(); this.store.snapshot();
@ -907,7 +731,7 @@ public class TestStore {
List<Cell> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family); List<Cell> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);
for(Cell kv : kvList2) { for(Cell kv : kvList2) {
this.store.add(KeyValueUtil.ensureKeyValue(kv)); this.store.add(kv, null);
} }
List<Cell> result; List<Cell> result;
@ -1049,7 +873,7 @@ public class TestStore {
assertEquals(0, this.store.getStorefilesCount()); assertEquals(0, this.store.getStorefilesCount());
// add some data, flush // add some data, flush
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
flush(1); flush(1);
assertEquals(1, this.store.getStorefilesCount()); assertEquals(1, this.store.getStorefilesCount());
@ -1097,7 +921,7 @@ public class TestStore {
assertEquals(0, this.store.getStorefilesCount()); assertEquals(0, this.store.getStorefilesCount());
// add some data, flush // add some data, flush
this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null)); this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null), null);
flush(1); flush(1);
// add one more file // add one more file
addStoreFile(); addStoreFile();

View File

@ -127,10 +127,10 @@ public class TestWalAndCompactingMemStoreFlush {
// Set up the configuration // Set up the configuration
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
FlushNonSloppyStoresFirstPolicy.class.getName()); FlushNonSloppyStoresFirstPolicy.class.getName());
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 200 * 1024); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25); conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25);
// set memstore to do data compaction // set memstore to do data compaction
conf.set("hbase.hregion.compacting.memstore.type", "data-compaction"); conf.set("hbase.hregion.compacting.memstore.type", "data-compaction");
@ -164,9 +164,9 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
@ -188,22 +188,18 @@ public class TestWalAndCompactingMemStoreFlush {
// Some other sanity checks. // Some other sanity checks.
assertTrue(smallestSeqCF1PhaseI < smallestSeqCF2PhaseI); assertTrue(smallestSeqCF1PhaseI < smallestSeqCF2PhaseI);
assertTrue(smallestSeqCF2PhaseI < smallestSeqCF3PhaseI); assertTrue(smallestSeqCF2PhaseI < smallestSeqCF3PhaseI);
assertTrue(cf1MemstoreSizePhaseI > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf2MemstoreSizePhaseI > 0); assertTrue(cf2MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf3MemstoreSizePhaseI > 0); assertTrue(cf3MemstoreSizePhaseI.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
String msg = "totalMemstoreSize="+totalMemstoreSize + String msg = "totalMemstoreSize="+totalMemstoreSize +
" DefaultMemStore.DEEP_OVERHEAD="+DefaultMemStore.DEEP_OVERHEAD +
" CompactingMemStore.DEEP_OVERHEAD="+CompactingMemStore.DEEP_OVERHEAD +
" cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI + " cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI +
" cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI + " cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI +
" cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ; " cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ;
assertEquals(msg, assertEquals(msg, totalMemstoreSize, cf1MemstoreSizePhaseI.getDataSize()
totalMemstoreSize + 2 * (CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD) + cf2MemstoreSizePhaseI.getDataSize() + cf3MemstoreSizePhaseI.getDataSize());
+ (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD),
cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI);
// Flush!!!!!!!!!!!!!!!!!!!!!! // Flush!!!!!!!!!!!!!!!!!!!!!!
// We have big compacting memstore CF1 and two small memstores: // We have big compacting memstore CF1 and two small memstores:
@ -219,9 +215,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -230,29 +226,21 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2);
long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3);
s = s + "DefaultMemStore DEEP_OVERHEAD is:" + DefaultMemStore.DEEP_OVERHEAD s = s + "\n----After first flush! CF1 should be flushed to memory, but not compacted.---\n"
+ ", CompactingMemStore DEEP_OVERHEAD is:" + CompactingMemStore.DEEP_OVERHEAD
+ "\n----After first flush! CF1 should be flushed to memory, but not compacted.---\n"
+ "Size of CF1 is:" + cf1MemstoreSizePhaseII + ", size of CF2 is:" + cf2MemstoreSizePhaseII + "Size of CF1 is:" + cf1MemstoreSizePhaseII + ", size of CF2 is:" + cf2MemstoreSizePhaseII
+ ", size of CF3 is:" + cf3MemstoreSizePhaseII + "\n"; + ", size of CF3 is:" + cf3MemstoreSizePhaseII + "\n";
// CF1 was flushed to memory, but there is nothing to compact, and CF! was flattened // CF1 was flushed to memory, but there is nothing to compact, and CF1 was flattened
assertTrue(cf1MemstoreSizePhaseII < cf1MemstoreSizePhaseI); assertTrue(cf1MemstoreSizePhaseII.getDataSize() == cf1MemstoreSizePhaseI.getDataSize());
assertTrue(cf1MemstoreSizePhaseII.getHeapOverhead() < cf1MemstoreSizePhaseI.getHeapOverhead());
// CF2 should become empty // CF2 should become empty
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseII);
cf2MemstoreSizePhaseII);
// verify that CF3 was flushed to memory and was compacted (this is approximation check) // verify that CF3 was flushed to memory and was compacted (this is approximation check)
assertTrue(cf3MemstoreSizePhaseI / 2 + CompactingMemStore.DEEP_OVERHEAD assertTrue(cf3MemstoreSizePhaseI.getDataSize() > cf3MemstoreSizePhaseII.getDataSize());
+ ImmutableSegment.DEEP_OVERHEAD_CAM assertTrue(
+ CompactionPipeline.ENTRY_OVERHEAD > cf3MemstoreSizePhaseII); cf3MemstoreSizePhaseI.getHeapOverhead() / 2 > cf3MemstoreSizePhaseII.getHeapOverhead());
// CF3 was compacted and flattened!
assertTrue("\n<<< Size of CF3 in phase I - " + cf3MemstoreSizePhaseI
+ ", size of CF3 in phase II - " + cf3MemstoreSizePhaseII + "\n",
cf3MemstoreSizePhaseI / 2 > cf3MemstoreSizePhaseII);
// Now the smallest LSN in the region should be the same as the smallest // Now the smallest LSN in the region should be the same as the smallest
// LSN in the memstore of CF1. // LSN in the memstore of CF1.
@ -270,7 +258,7 @@ public class TestWalAndCompactingMemStoreFlush {
+ smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n"; + smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n";
// How much does the CF1 memstore occupy? Will be used later. // How much does the CF1 memstore occupy? Will be used later.
long cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore();
long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1);
s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII
@ -284,9 +272,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -306,9 +294,8 @@ public class TestWalAndCompactingMemStoreFlush {
// CF1's pipeline component (inserted before first flush) should be flushed to disk // CF1's pipeline component (inserted before first flush) should be flushed to disk
// CF2 should be flushed to disk // CF2 should be flushed to disk
assertTrue(cf1MemstoreSizePhaseIII > cf1MemstoreSizePhaseIV); assertTrue(cf1MemstoreSizePhaseIII.getDataSize() > cf1MemstoreSizePhaseIV.getDataSize());
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseIV);
cf2MemstoreSizePhaseIV);
// CF3 shouldn't have been touched. // CF3 shouldn't have been touched.
assertEquals(cf3MemstoreSizePhaseIV, cf3MemstoreSizePhaseII); assertEquals(cf3MemstoreSizePhaseIV, cf3MemstoreSizePhaseII);
@ -322,34 +309,25 @@ public class TestWalAndCompactingMemStoreFlush {
// Flush!!!!!!!!!!!!!!!!!!!!!! // Flush!!!!!!!!!!!!!!!!!!!!!!
// Trying to clean the existing memstores, CF2 all flushed to disk. The single // Trying to clean the existing memstores, CF2 all flushed to disk. The single
// memstore segment in the compaction pipeline of CF1 and CF3 should be flushed to disk. // memstore segment in the compaction pipeline of CF1 and CF3 should be flushed to disk.
// Note that active set of CF3 is empty
// But active set of CF1 is not yet empty
region.flush(true); region.flush(true);
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
assertTrue( assertEquals(MemstoreSize.EMPTY_SIZE , cf1MemstoreSizePhaseV);
CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD <= cf1MemstoreSizePhaseV); assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseV);
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf3MemstoreSizePhaseV);
cf2MemstoreSizePhaseV);
assertEquals(CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD,
cf3MemstoreSizePhaseV);
region.flush(true); // flush once again in order to be sure that everything is empty
assertEquals(CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD,
region.getStore(FAMILY1).getMemStoreSize());
// What happens when we hit the memstore limit, but we are not able to find // What happens when we hit the memstore limit, but we are not able to find
// any Column Family above the threshold? // any Column Family above the threshold?
// In that case, we should flush all the CFs. // In that case, we should flush all the CFs.
// The memstore limit is 200*1024 and the column family flush threshold is // The memstore limit is 100*1024 and the column family flush threshold is
// around 50*1024. We try to just hit the memstore limit with each CF's // around 25*1024. We try to just hit the memstore limit with each CF's
// memstore being below the CF flush threshold. // memstore being below the CF flush threshold.
for (int i = 1; i <= 300; i++) { for (int i = 1; i <= 300; i++) {
region.put(createPut(1, i)); region.put(createPut(1, i));
@ -384,10 +362,10 @@ public class TestWalAndCompactingMemStoreFlush {
/* SETUP */ /* SETUP */
// Set up the configuration // Set up the configuration
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY,
FlushNonSloppyStoresFirstPolicy.class.getName()); FlushNonSloppyStoresFirstPolicy.class.getName());
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 200 * 1024); conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 * 1024);
conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5); conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
// set memstore to index-compaction // set memstore to index-compaction
conf.set("hbase.hregion.compacting.memstore.type", "index-compaction"); conf.set("hbase.hregion.compacting.memstore.type", "index-compaction");
@ -421,9 +399,9 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2);
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -436,18 +414,14 @@ public class TestWalAndCompactingMemStoreFlush {
// Some other sanity checks. // Some other sanity checks.
assertTrue(smallestSeqCF1PhaseI < smallestSeqCF2PhaseI); assertTrue(smallestSeqCF1PhaseI < smallestSeqCF2PhaseI);
assertTrue(smallestSeqCF2PhaseI < smallestSeqCF3PhaseI); assertTrue(smallestSeqCF2PhaseI < smallestSeqCF3PhaseI);
assertTrue(cf1MemstoreSizePhaseI > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf2MemstoreSizePhaseI > 0); assertTrue(cf2MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf3MemstoreSizePhaseI > 0); assertTrue(cf3MemstoreSizePhaseI.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
assertEquals( assertEquals(totalMemstoreSizePhaseI, cf1MemstoreSizePhaseI.getDataSize()
totalMemstoreSizePhaseI + cf2MemstoreSizePhaseI.getDataSize() + cf3MemstoreSizePhaseI.getDataSize());
+ 1 * DefaultMemStore.DEEP_OVERHEAD
+ 2 * CompactingMemStore.DEEP_OVERHEAD
+ 3 * MutableSegment.DEEP_OVERHEAD,
cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI);
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE I - Flush */ /* PHASE I - Flush */
@ -475,9 +449,9 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE II - collect sizes */ /* PHASE II - collect sizes */
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
// Find the smallest LSNs for edits wrt to each CF. // Find the smallest LSNs for edits wrt to each CF.
@ -487,13 +461,15 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE II - validation */ /* PHASE II - validation */
// CF1 was flushed to memory, should be flattened and take less space // CF1 was flushed to memory, should be flattened and take less space
assertTrue(cf1MemstoreSizePhaseII < cf1MemstoreSizePhaseI); assertEquals(cf1MemstoreSizePhaseII.getDataSize() , cf1MemstoreSizePhaseI.getDataSize());
assertTrue(cf1MemstoreSizePhaseII.getHeapOverhead() < cf1MemstoreSizePhaseI.getHeapOverhead());
// CF2 should become empty // CF2 should become empty
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseII);
cf2MemstoreSizePhaseII);
// verify that CF3 was flushed to memory and was not compacted (this is an approximation check) // verify that CF3 was flushed to memory and was not compacted (this is an approximation check)
// if compacted CF# should be at least twice less because its every key was duplicated // if compacted CF# should be at least twice less because its every key was duplicated
assertTrue(cf3MemstoreSizePhaseI / 2 < cf3MemstoreSizePhaseII); assertEquals(cf3MemstoreSizePhaseII.getDataSize() , cf3MemstoreSizePhaseI.getDataSize());
assertTrue(
cf3MemstoreSizePhaseI.getHeapOverhead() / 2 < cf3MemstoreSizePhaseII.getHeapOverhead());
// Now the smallest LSN in the region should be the same as the smallest // Now the smallest LSN in the region should be the same as the smallest
// LSN in the memstore of CF1. // LSN in the memstore of CF1.
@ -501,14 +477,8 @@ public class TestWalAndCompactingMemStoreFlush {
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. Counting the empty active segments in CF1/2/3 and pipeline // memstores of CF1, CF2 and CF3. Counting the empty active segments in CF1/2/3 and pipeline
// items in CF1/2 // items in CF1/2
assertEquals( assertEquals(totalMemstoreSizePhaseII, cf1MemstoreSizePhaseII.getDataSize()
totalMemstoreSizePhaseII + cf2MemstoreSizePhaseII.getDataSize() + cf3MemstoreSizePhaseII.getDataSize());
+ 1 * DefaultMemStore.DEEP_OVERHEAD
+ 2 * CompactingMemStore.DEEP_OVERHEAD
+ 3 * MutableSegment.DEEP_OVERHEAD
+ 2 * CompactionPipeline.ENTRY_OVERHEAD
+ 2 * ImmutableSegment.DEEP_OVERHEAD_CAM,
cf1MemstoreSizePhaseII + cf2MemstoreSizePhaseII + cf3MemstoreSizePhaseII);
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
@ -528,7 +498,7 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE III - collect sizes */ /* PHASE III - collect sizes */
// How much does the CF1 memstore occupy now? Will be used later. // How much does the CF1 memstore occupy now? Will be used later.
long cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore();
long totalMemstoreSizePhaseIII = region.getMemstoreSize(); long totalMemstoreSizePhaseIII = region.getMemstoreSize();
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
@ -536,14 +506,8 @@ public class TestWalAndCompactingMemStoreFlush {
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. Counting the empty active segments in CF1/2/3 and pipeline // memstores of CF1, CF2 and CF3. Counting the empty active segments in CF1/2/3 and pipeline
// items in CF1/2 // items in CF1/2
assertEquals( assertEquals(totalMemstoreSizePhaseIII, cf1MemstoreSizePhaseIII.getDataSize()
totalMemstoreSizePhaseIII + cf2MemstoreSizePhaseII.getDataSize() + cf3MemstoreSizePhaseII.getDataSize());
+ 1 * DefaultMemStore.DEEP_OVERHEAD
+ 2 * CompactingMemStore.DEEP_OVERHEAD
+ 3 * MutableSegment.DEEP_OVERHEAD
+ 2 * CompactionPipeline.ENTRY_OVERHEAD
+ 2 * ImmutableSegment.DEEP_OVERHEAD_CAM,
cf1MemstoreSizePhaseIII + cf2MemstoreSizePhaseII + cf3MemstoreSizePhaseII);
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE III - Flush */ /* PHASE III - Flush */
@ -556,9 +520,9 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE IV - collect sizes */ /* PHASE IV - collect sizes */
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3);
@ -567,9 +531,8 @@ public class TestWalAndCompactingMemStoreFlush {
/* PHASE IV - validation */ /* PHASE IV - validation */
// CF1's biggest pipeline component (inserted before first flush) should be flushed to disk // CF1's biggest pipeline component (inserted before first flush) should be flushed to disk
// CF2 should remain empty // CF2 should remain empty
assertTrue(cf1MemstoreSizePhaseIII > cf1MemstoreSizePhaseIV); assertTrue(cf1MemstoreSizePhaseIII.getDataSize() > cf1MemstoreSizePhaseIV.getDataSize());
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseIV);
cf2MemstoreSizePhaseIV);
// CF3 shouldn't have been touched. // CF3 shouldn't have been touched.
assertEquals(cf3MemstoreSizePhaseIV, cf3MemstoreSizePhaseII); assertEquals(cf3MemstoreSizePhaseIV, cf3MemstoreSizePhaseII);
// the smallest LSN of CF3 shouldn't change // the smallest LSN of CF3 shouldn't change
@ -588,23 +551,20 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE V - collect sizes */ /* PHASE V - collect sizes */
// Recalculate everything // Recalculate everything
long cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
long totalMemstoreSizePhaseV = region.getMemstoreSize(); long totalMemstoreSizePhaseV = region.getMemstoreSize();
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE V - validation */ /* PHASE V - validation */
assertEquals(CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf1MemstoreSizePhaseV);
cf1MemstoreSizePhaseV); assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseV);
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf3MemstoreSizePhaseV);
cf2MemstoreSizePhaseV);
assertEquals(CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD,
cf3MemstoreSizePhaseV);
// The total memstores size should be empty // The total memstores size should be empty
assertEquals(totalMemstoreSizePhaseV, 0); assertEquals(0, totalMemstoreSizePhaseV);
// Because there is nothing in any memstore the WAL's LSN should be -1 // Because there is nothing in any memstore the WAL's LSN should be -1
assertEquals(smallestSeqInRegionCurrentMemstorePhaseV, HConstants.NO_SEQNUM); assertEquals(smallestSeqInRegionCurrentMemstorePhaseV, HConstants.NO_SEQNUM);
@ -626,9 +586,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.put(createPut(5, i)); region.put(createPut(5, i));
} }
long cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getSizeOfMemStore();
long cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getSizeOfMemStore();
long cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getMemStoreSize(); MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getSizeOfMemStore();
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE VI - Flush */ /* PHASE VI - Flush */
@ -639,13 +599,13 @@ public class TestWalAndCompactingMemStoreFlush {
// Since we won't find any CF above the threshold, and hence no specific // Since we won't find any CF above the threshold, and hence no specific
// store to flush, we should flush all the memstores // store to flush, we should flush all the memstores
// Also compacted memstores are flushed to disk, but not entirely emptied // Also compacted memstores are flushed to disk, but not entirely emptied
long cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getSizeOfMemStore();
long cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getSizeOfMemStore();
long cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getMemStoreSize(); MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getSizeOfMemStore();
assertTrue(cf1ActiveSizePhaseVII < cf1ActiveSizePhaseVI); assertTrue(cf1ActiveSizePhaseVII.getDataSize() < cf1ActiveSizePhaseVI.getDataSize());
assertTrue(cf3ActiveSizePhaseVII < cf3ActiveSizePhaseVI); assertTrue(cf3ActiveSizePhaseVII.getDataSize() < cf3ActiveSizePhaseVI.getDataSize());
assertTrue(cf5ActiveSizePhaseVII < cf5ActiveSizePhaseVI); assertTrue(cf5ActiveSizePhaseVII.getDataSize() < cf5ActiveSizePhaseVI.getDataSize());
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }
@ -654,10 +614,10 @@ public class TestWalAndCompactingMemStoreFlush {
public void testSelectiveFlushAndWALinDataCompaction() throws IOException { public void testSelectiveFlushAndWALinDataCompaction() throws IOException {
// Set up the configuration // Set up the configuration
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 600 * 1024); conf.setLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 300 * 1024);
conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushNonSloppyStoresFirstPolicy.class
.getName()); .getName());
conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 200 * conf.setLong(FlushLargeStoresPolicy.HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, 75 *
1024); 1024);
conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5); conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.5);
// set memstore to do data compaction and not to use the speculative scan // set memstore to do data compaction and not to use the speculative scan
@ -683,14 +643,14 @@ public class TestWalAndCompactingMemStoreFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSizePhaseI > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf2MemstoreSizePhaseI > 0); assertTrue(cf2MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf3MemstoreSizePhaseI > 0); assertTrue(cf3MemstoreSizePhaseI.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
@ -699,10 +659,8 @@ public class TestWalAndCompactingMemStoreFlush {
" cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI + " cf1MemstoreSizePhaseI="+cf1MemstoreSizePhaseI +
" cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI + " cf2MemstoreSizePhaseI="+cf2MemstoreSizePhaseI +
" cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ; " cf3MemstoreSizePhaseI="+cf3MemstoreSizePhaseI ;
assertEquals(msg, assertEquals(msg, totalMemstoreSize, cf1MemstoreSizePhaseI.getDataSize()
totalMemstoreSize + 2 * (CompactingMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD) + cf2MemstoreSizePhaseI.getDataSize() + cf3MemstoreSizePhaseI.getDataSize());
+ (DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD),
cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI);
// Flush! // Flush!
CompactingMemStore cms1 = (CompactingMemStore) ((HStore) region.getStore(FAMILY1)).memstore; CompactingMemStore cms1 = (CompactingMemStore) ((HStore) region.getStore(FAMILY1)).memstore;
@ -711,7 +669,7 @@ public class TestWalAndCompactingMemStoreFlush {
cms3.flushInMemory(); cms3.flushInMemory();
region.flush(false); region.flush(false);
long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseII = long smallestSeqInRegionCurrentMemstorePhaseII =
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -720,8 +678,7 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3);
// CF2 should have been cleared // CF2 should have been cleared
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseII);
cf2MemstoreSizePhaseII);
String s = "\n\n----------------------------------\n" String s = "\n\n----------------------------------\n"
+ "Upon initial insert and flush, LSN of CF1 is:" + "Upon initial insert and flush, LSN of CF1 is:"
@ -816,23 +773,19 @@ public class TestWalAndCompactingMemStoreFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
long cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
long cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
long cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSizePhaseI > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf2MemstoreSizePhaseI > 0); assertTrue(cf2MemstoreSizePhaseI.getDataSize() > 0);
assertTrue(cf3MemstoreSizePhaseI > 0); assertTrue(cf3MemstoreSizePhaseI.getDataSize() > 0);
// The total memstore size should be the same as the sum of the sizes of // The total memstore size should be the same as the sum of the sizes of
// memstores of CF1, CF2 and CF3. // memstores of CF1, CF2 and CF3.
assertEquals( assertEquals(totalMemstoreSize, cf1MemstoreSizePhaseI.getDataSize()
totalMemstoreSize + cf2MemstoreSizePhaseI.getDataSize() + cf3MemstoreSizePhaseI.getDataSize());
+ 1 * DefaultMemStore.DEEP_OVERHEAD
+ 2 * CompactingMemStore.DEEP_OVERHEAD
+ 3 * MutableSegment.DEEP_OVERHEAD,
cf1MemstoreSizePhaseI + cf2MemstoreSizePhaseI + cf3MemstoreSizePhaseI);
// Flush! // Flush!
((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory(); ((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory();
@ -848,7 +801,7 @@ public class TestWalAndCompactingMemStoreFlush {
} }
region.flush(false); region.flush(false);
long cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
long smallestSeqInRegionCurrentMemstorePhaseII = region.getWAL() long smallestSeqInRegionCurrentMemstorePhaseII = region.getWAL()
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -857,8 +810,7 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3);
// CF2 should have been cleared // CF2 should have been cleared
assertEquals(DefaultMemStore.DEEP_OVERHEAD + MutableSegment.DEEP_OVERHEAD, assertEquals(MemstoreSize.EMPTY_SIZE, cf2MemstoreSizePhaseII);
cf2MemstoreSizePhaseII);
// Add same entries to compact them later // Add same entries to compact them later
for (int i = 1; i <= 1200; i++) { for (int i = 1; i <= 1200; i++) {

View File

@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
import org.apache.hadoop.hbase.regionserver.MemstoreSize;
import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner;
@ -550,7 +551,7 @@ public abstract class AbstractTestWALReplay {
final Configuration newConf = HBaseConfiguration.create(this.conf); final Configuration newConf = HBaseConfiguration.create(this.conf);
User user = HBaseTestingUtility.getDifferentUser(newConf, User user = HBaseTestingUtility.getDifferentUser(newConf,
tableName.getNameAsString()); tableName.getNameAsString());
user.runAs(new PrivilegedExceptionAction() { user.runAs(new PrivilegedExceptionAction<Object>() {
@Override @Override
public Object run() throws Exception { public Object run() throws Exception {
runWALSplit(newConf); runWALSplit(newConf);
@ -560,10 +561,9 @@ public abstract class AbstractTestWALReplay {
final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); final AtomicInteger countOfRestoredEdits = new AtomicInteger(0);
HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) { HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) {
@Override @Override
protected boolean restoreEdit(HStore s, Cell cell) { protected void restoreEdit(HStore s, Cell cell, MemstoreSize memstoreSize) {
boolean b = super.restoreEdit(s, cell); super.restoreEdit(s, cell, memstoreSize);
countOfRestoredEdits.incrementAndGet(); countOfRestoredEdits.incrementAndGet();
return b;
} }
}; };
long seqid3 = region3.initialize(); long seqid3 = region3.initialize();