HBASE-15786 Create DBB backed MSLAB pool.
This commit is contained in:
parent
a45b3c3533
commit
86e17858f7
|
@ -22,7 +22,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -91,9 +91,9 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
|
||||
private static int compareQualifierPart(Cell cell, int length, byte[] prefix) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), length, prefix, 0, length);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), length, prefix, 0, length);
|
||||
}
|
||||
return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0,
|
||||
length);
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
@ -62,8 +62,8 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
|
||||
private Cell createKeyOnlyCell(Cell c) {
|
||||
if (c instanceof ByteBufferedCell) {
|
||||
return new KeyOnlyByteBufferedCell((ByteBufferedCell) c, lenAsVal);
|
||||
if (c instanceof ByteBufferCell) {
|
||||
return new KeyOnlyByteBufferCell((ByteBufferCell) c, lenAsVal);
|
||||
} else {
|
||||
return new KeyOnlyCell(c, lenAsVal);
|
||||
}
|
||||
|
@ -232,11 +232,11 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
static class KeyOnlyByteBufferedCell extends ByteBufferedCell {
|
||||
private ByteBufferedCell cell;
|
||||
static class KeyOnlyByteBufferCell extends ByteBufferCell {
|
||||
private ByteBufferCell cell;
|
||||
private boolean lenAsVal;
|
||||
|
||||
public KeyOnlyByteBufferedCell(ByteBufferedCell c, boolean lenAsVal) {
|
||||
public KeyOnlyByteBufferCell(ByteBufferCell c, boolean lenAsVal) {
|
||||
this.cell = c;
|
||||
this.lenAsVal = lenAsVal;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
|
@ -62,9 +62,9 @@ public class PrefixFilter extends FilterBase {
|
|||
// else return true, filter row
|
||||
// if we are passed the prefix, set flag
|
||||
int cmp;
|
||||
if (firstRowCell instanceof ByteBufferedCell) {
|
||||
cmp = ByteBufferUtils.compareTo(((ByteBufferedCell) firstRowCell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) firstRowCell).getRowPosition(), this.prefix.length,
|
||||
if (firstRowCell instanceof ByteBufferCell) {
|
||||
cmp = ByteBufferUtils.compareTo(((ByteBufferCell) firstRowCell).getRowByteBuffer(),
|
||||
((ByteBufferCell) firstRowCell).getRowPosition(), this.prefix.length,
|
||||
this.prefix, 0, this.prefix.length);
|
||||
} else {
|
||||
cmp = Bytes.compareTo(firstRowCell.getRowArray(), firstRowCell.getRowOffset(),
|
||||
|
|
|
@ -25,7 +25,7 @@ import java.nio.ByteBuffer;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -51,18 +51,18 @@ public class TestComparators {
|
|||
// Row compare
|
||||
KeyValue kv = new KeyValue(r1, f, q1, v1);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
ByteArrayComparable comparable = new BinaryComparator(r1);
|
||||
assertEquals(0, CellComparator.compareRow(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareRow(kv, comparable));
|
||||
kv = new KeyValue(r0, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareRow(bbCell, comparable) > 0);
|
||||
assertTrue(CellComparator.compareRow(kv, comparable) > 0);
|
||||
kv = new KeyValue(r2, f, q1, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareRow(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareRow(kv, comparable) < 0);
|
||||
// Qualifier compare
|
||||
|
@ -71,12 +71,12 @@ public class TestComparators {
|
|||
assertEquals(0, CellComparator.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q2, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareQualifier(bbCell, comparable));
|
||||
assertEquals(0, CellComparator.compareQualifier(kv, comparable));
|
||||
kv = new KeyValue(r2, f, q3, v1);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareQualifier(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareQualifier(kv, comparable) < 0);
|
||||
// Value compare
|
||||
|
@ -85,7 +85,7 @@ public class TestComparators {
|
|||
assertEquals(0, CellComparator.compareValue(kv, comparable));
|
||||
kv = new KeyValue(r1, f, q1, v2);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellComparator.compareValue(bbCell, comparable) < 0);
|
||||
assertTrue(CellComparator.compareValue(kv, comparable) < 0);
|
||||
// Family compare
|
||||
|
|
|
@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyByteBufferCell;
|
||||
import org.apache.hadoop.hbase.filter.KeyOnlyFilter.KeyOnlyCell;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -71,7 +71,7 @@ public class TestKeyOnlyFilter {
|
|||
v.length, tags);
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
ByteBufferedCellImpl bbCell = new ByteBufferedCellImpl(buffer, 0,
|
||||
ByteBufferCellImpl bbCell = new ByteBufferCellImpl(buffer, 0,
|
||||
buffer.remaining());
|
||||
|
||||
// KV format: <keylen:4><valuelen:4><key:keylen><value:valuelen>
|
||||
|
@ -89,7 +89,7 @@ public class TestKeyOnlyFilter {
|
|||
KeyValue KeyOnlyKeyValue = new KeyValue(newBuffer);
|
||||
|
||||
KeyOnlyCell keyOnlyCell = new KeyOnlyCell(kv, lenAsVal);
|
||||
KeyOnlyByteBufferedCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferedCell(
|
||||
KeyOnlyByteBufferCell keyOnlyByteBufferedCell = new KeyOnlyByteBufferCell(
|
||||
bbCell, lenAsVal);
|
||||
|
||||
assertTrue(CellUtil.matchingRows(KeyOnlyKeyValue, keyOnlyCell));
|
||||
|
|
|
@ -44,8 +44,30 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
* appropriate Cell access server-side: i.e. ByteBufferedCell when backed by a ByteBuffer and Cell
|
||||
* when it is not.
|
||||
*/
|
||||
/*
|
||||
* Even though all the methods are abstract, ByteBufferCell is not made to be an interface with
|
||||
* intent. In CellComparator compare method, we have instance of check to decide whether to use
|
||||
* getXXXArray() or getXXXByteBuffer(). This is a very hot method in read and write paths.
|
||||
* if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
....
|
||||
}
|
||||
if (left instanceof ByteBufferCell) {
|
||||
....
|
||||
}
|
||||
if (right instanceof ByteBufferCell) {
|
||||
....
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
* We did JMH micro benchmark tests with both left and right cells as ByteBufferCell, one only
|
||||
* ByteBufferCell and both as Cells. This is compared against JMH results on compare logic with out
|
||||
* any instance of checks. We noticed that if ByteBufferCell is an interface, the benchmark result
|
||||
* seems to be very bad for case of both right and left are Cell only (Not ByteBufferCell). When
|
||||
* ByteBufferCell is an abstract class all 4 possible cases giving almost similar performance number
|
||||
* compared with compare logic with no instance of checks.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract class ByteBufferedCell implements Cell {
|
||||
public abstract class ByteBufferCell implements Cell {
|
||||
/**
|
||||
* @return The {@link ByteBuffer} containing the row bytes.
|
||||
*/
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
* (onheap and offheap).
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ByteBufferedKeyOnlyKeyValue extends ByteBufferedCell {
|
||||
public class ByteBufferKeyOnlyKeyValue extends ByteBufferCell {
|
||||
|
||||
private ByteBuffer buf;
|
||||
private int offset = 0; // offset into buffer where key starts at
|
||||
|
@ -41,10 +41,10 @@ public class ByteBufferedKeyOnlyKeyValue extends ByteBufferedCell {
|
|||
* Used in cases where we want to avoid lot of garbage by allocating new objects with different
|
||||
* keys. Use the emtpy construtor and set the keys using {@link #setKey(ByteBuffer, int, int)}
|
||||
*/
|
||||
public ByteBufferedKeyOnlyKeyValue() {
|
||||
public ByteBufferKeyOnlyKeyValue() {
|
||||
}
|
||||
|
||||
public ByteBufferedKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
public ByteBufferKeyOnlyKeyValue(ByteBuffer buf, int offset, int length) {
|
||||
setKey(buf, offset, length);
|
||||
}
|
||||
|
|
@ -159,35 +159,35 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareFamilies(Cell left, Cell right) {
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), right.getFamilyLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
if (right instanceof ByteBufferCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(
|
||||
left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
((ByteBufferedCell)right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell)right).getFamilyPosition(), right.getFamilyLength());
|
||||
((ByteBufferCell)right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell)right).getFamilyPosition(), right.getFamilyLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
right.getFamilyArray(), right.getFamilyOffset(), right.getFamilyLength());
|
||||
}
|
||||
|
||||
private final static int compareFamilies(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(), right,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(),
|
||||
|
@ -201,28 +201,28 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return 0 if both cells are equal, 1 if left cell is bigger than right, -1 otherwise
|
||||
*/
|
||||
public final static int compareQualifiers(Cell left, Cell right) {
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils
|
||||
.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(),
|
||||
left.getQualifierLength(), ((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPosition(),
|
||||
.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(),
|
||||
left.getQualifierLength(), ((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(),
|
||||
right.getQualifierLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
right.getQualifierArray(), right.getQualifierOffset(), right.getQualifierLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
if (right instanceof ByteBufferCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(left.getQualifierArray(),
|
||||
left.getQualifierOffset(), left.getQualifierLength(),
|
||||
((ByteBufferedCell)right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell)right).getQualifierPosition(), right.getQualifierLength());
|
||||
((ByteBufferCell)right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell)right).getQualifierPosition(), right.getQualifierLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
left.getQualifierLength(), right.getQualifierArray(), right.getQualifierOffset(),
|
||||
|
@ -230,9 +230,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
}
|
||||
|
||||
public final static int compareQualifiers(Cell left, byte[] right, int rOffset, int rLength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
right, rOffset, rLength);
|
||||
}
|
||||
return Bytes.compareTo(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
|
@ -328,25 +328,25 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
if (left == right) {
|
||||
return 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPosition(), right.getRowLength());
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
|
||||
((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), right.getRowLength());
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), left.getRowLength(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
if (right instanceof ByteBufferCell) {
|
||||
// Notice how we flip the order of the compare here. We used to negate the return value but
|
||||
// see what FindBugs says
|
||||
// http://findbugs.sourceforge.net/bugDescriptions.html#RV_NEGATING_RESULT_OF_COMPARETO
|
||||
// It suggest flipping the order to get same effect and 'safer'.
|
||||
return ByteBufferUtils.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
((ByteBufferedCell)right).getRowByteBuffer(),
|
||||
((ByteBufferedCell)right).getRowPosition(), right.getRowLength());
|
||||
((ByteBufferCell)right).getRowByteBuffer(),
|
||||
((ByteBufferCell)right).getRowPosition(), right.getRowLength());
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(),
|
||||
right.getRowArray(), right.getRowOffset(), right.getRowLength());
|
||||
|
@ -369,9 +369,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* than byte[], -1 otherwise
|
||||
*/
|
||||
public int compareRows(Cell left, byte[] right, int roffset, int rlength) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), left.getRowLength(), right,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(), right,
|
||||
roffset, rlength);
|
||||
}
|
||||
return Bytes.compareTo(left.getRowArray(), left.getRowOffset(), left.getRowLength(), right,
|
||||
|
@ -521,9 +521,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return result comparing cell's row
|
||||
*/
|
||||
public static int compareRow(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
@ -535,9 +535,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return result comparing cell's column family
|
||||
*/
|
||||
public static int compareFamily(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getFamilyArray(), cell.getFamilyOffset(),
|
||||
cell.getFamilyLength());
|
||||
|
@ -550,9 +550,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return result comparing cell's qualifier
|
||||
*/
|
||||
public static int compareQualifier(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
|
||||
cell.getQualifierLength());
|
||||
|
@ -565,9 +565,9 @@ public class CellComparator implements Comparator<Cell>, Serializable {
|
|||
* @return result comparing cell's value
|
||||
*/
|
||||
public static int compareValue(Cell cell, ByteArrayComparable comparator) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return comparator.compareTo(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition(), cell.getValueLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return comparator.compareTo(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
|
||||
}
|
||||
return comparator.compareTo(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
|
|
@ -131,10 +131,10 @@ public final class CellUtil {
|
|||
|
||||
public static int copyRowTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
short rowLen = cell.getRowLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getRowArray(), cell.getRowOffset(), destination, destinationOffset,
|
||||
rowLen);
|
||||
|
@ -142,16 +142,28 @@ public final class CellUtil {
|
|||
return destinationOffset + rowLen;
|
||||
}
|
||||
|
||||
public static int copyRowTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
short rowLen = cell.getRowLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getRowPosition(), destinationOffset, rowLen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getRowArray(),
|
||||
cell.getRowOffset(), rowLen);
|
||||
}
|
||||
return destinationOffset + rowLen;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the row to a new byte[]
|
||||
* @param cell the cell from which row has to copied
|
||||
* @return the byte[] containing the row
|
||||
*/
|
||||
public static byte[] copyRow(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.copyOfRange(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(),
|
||||
((ByteBufferedCell) cell).getRowPosition() + cell.getRowLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.copyOfRange(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(),
|
||||
((ByteBufferCell) cell).getRowPosition() + cell.getRowLength());
|
||||
} else {
|
||||
return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowOffset() + cell.getRowLength());
|
||||
|
@ -160,10 +172,10 @@ public final class CellUtil {
|
|||
|
||||
public static int copyFamilyTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
byte fLen = cell.getFamilyLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
} else {
|
||||
System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(), destination,
|
||||
destinationOffset, fLen);
|
||||
|
@ -171,12 +183,24 @@ public final class CellUtil {
|
|||
return destinationOffset + fLen;
|
||||
}
|
||||
|
||||
public static int copyFamilyTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
byte fLen = cell.getFamilyLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getFamilyPosition(), destinationOffset, fLen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getFamilyArray(),
|
||||
cell.getFamilyOffset(), fLen);
|
||||
}
|
||||
return destinationOffset + fLen;
|
||||
}
|
||||
|
||||
public static int copyQualifierTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int qlen = cell.getQualifierLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), destination,
|
||||
destinationOffset, qlen);
|
||||
|
@ -184,12 +208,24 @@ public final class CellUtil {
|
|||
return destinationOffset + qlen;
|
||||
}
|
||||
|
||||
public static int copyQualifierTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int qlen = cell.getQualifierLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getQualifierPosition(), destinationOffset, qlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset,
|
||||
cell.getQualifierArray(), cell.getQualifierOffset(), qlen);
|
||||
}
|
||||
return destinationOffset + qlen;
|
||||
}
|
||||
|
||||
public static int copyValueTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int vlen = cell.getValueLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getValueArray(), cell.getValueOffset(), destination, destinationOffset,
|
||||
vlen);
|
||||
|
@ -197,6 +233,18 @@ public final class CellUtil {
|
|||
return destinationOffset + vlen;
|
||||
}
|
||||
|
||||
public static int copyValueTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int vlen = cell.getValueLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getValuePosition(), destinationOffset, vlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getValueArray(),
|
||||
cell.getValueOffset(), vlen);
|
||||
}
|
||||
return destinationOffset + vlen;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the tags info into the tag portion of the cell
|
||||
* @param cell
|
||||
|
@ -206,10 +254,10 @@ public final class CellUtil {
|
|||
*/
|
||||
public static int copyTagTo(Cell cell, byte[] destination, int destinationOffset) {
|
||||
int tlen = cell.getTagsLength();
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToArray(destination,
|
||||
((ByteBufferedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferedCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
} else {
|
||||
System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
|
||||
tlen);
|
||||
|
@ -217,22 +265,34 @@ public final class CellUtil {
|
|||
return destinationOffset + tlen;
|
||||
}
|
||||
|
||||
public static int copyTagTo(Cell cell, ByteBuffer destination, int destinationOffset) {
|
||||
int tlen = cell.getTagsLength();
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
|
||||
} else {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
|
||||
cell.getTagsOffset(), tlen);
|
||||
}
|
||||
return destinationOffset + tlen;
|
||||
}
|
||||
|
||||
/********************* misc *************************************/
|
||||
|
||||
@Private
|
||||
public static byte getRowByte(Cell cell, int index) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ((ByteBufferedCell) cell).getRowByteBuffer().get(
|
||||
((ByteBufferedCell) cell).getRowPosition() + index);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) cell).getRowByteBuffer().get(
|
||||
((ByteBufferCell) cell).getRowPosition() + index);
|
||||
}
|
||||
return cell.getRowArray()[cell.getRowOffset() + index];
|
||||
}
|
||||
|
||||
@Private
|
||||
public static byte getQualifierByte(Cell cell, int index) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ((ByteBufferedCell) cell).getQualifierByteBuffer().get(
|
||||
((ByteBufferedCell) cell).getQualifierPosition() + index);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ((ByteBufferCell) cell).getQualifierByteBuffer().get(
|
||||
((ByteBufferCell) cell).getQualifierPosition() + index);
|
||||
}
|
||||
return cell.getQualifierArray()[cell.getQualifierOffset() + index];
|
||||
}
|
||||
|
@ -546,12 +606,12 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] buf, int offset) {
|
||||
offset = KeyValueUtil.appendToByteArray(this.cell, buf, offset, false);
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
offset = KeyValueUtil.appendToByteBuffer(this.cell, buf, offset, false);
|
||||
int tagsLen = this.tags.length;
|
||||
assert tagsLen > 0;
|
||||
offset = Bytes.putAsShort(buf, offset, tagsLen);
|
||||
System.arraycopy(this.tags, 0, buf, offset, tagsLen);
|
||||
offset = ByteBufferUtils.putAsShort(buf, offset, tagsLen);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.tags, 0, tagsLen);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -720,9 +780,9 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingRow(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), left.getRowLength(), buf, offset,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), left.getRowLength(), buf, offset,
|
||||
length);
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), left.getRowLength(), buf, offset,
|
||||
|
@ -732,20 +792,20 @@ public final class CellUtil {
|
|||
public static boolean matchingFamily(final Cell left, final Cell right) {
|
||||
byte lfamlength = left.getFamilyLength();
|
||||
byte rfamlength = right.getFamilyLength();
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), lfamlength,
|
||||
((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPosition(), rfamlength);
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), lfamlength,
|
||||
((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), rfamlength);
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), lfamlength,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), lfamlength,
|
||||
right.getFamilyArray(), right.getFamilyOffset(), rfamlength);
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) right).getFamilyPosition(), rfamlength,
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) right).getFamilyPosition(), rfamlength,
|
||||
left.getFamilyArray(), left.getFamilyOffset(), lfamlength);
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), lfamlength,
|
||||
|
@ -761,9 +821,9 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingFamily(final Cell left, final byte[] buf, final int offset,
|
||||
final int length) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) left).getFamilyPosition(), left.getFamilyLength(), buf,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) left).getFamilyPosition(), left.getFamilyLength(), buf,
|
||||
offset, length);
|
||||
}
|
||||
return Bytes.equals(left.getFamilyArray(), left.getFamilyOffset(), left.getFamilyLength(), buf,
|
||||
|
@ -773,20 +833,20 @@ public final class CellUtil {
|
|||
public static boolean matchingQualifier(final Cell left, final Cell right) {
|
||||
int lqlength = left.getQualifierLength();
|
||||
int rqlength = right.getQualifierLength();
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(), lqlength,
|
||||
((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPosition(), rqlength);
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), lqlength,
|
||||
((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(), rqlength);
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(), lqlength,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), lqlength,
|
||||
right.getQualifierArray(), right.getQualifierOffset(), rqlength);
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) right).getQualifierPosition(), rqlength,
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) right).getQualifierPosition(), rqlength,
|
||||
left.getQualifierArray(), left.getQualifierOffset(), lqlength);
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
|
@ -822,9 +882,9 @@ public final class CellUtil {
|
|||
if (buf == null) {
|
||||
return left.getQualifierLength() == 0;
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) left).getQualifierPosition(), left.getQualifierLength(),
|
||||
buf, offset, length);
|
||||
}
|
||||
return Bytes.equals(left.getQualifierArray(), left.getQualifierOffset(),
|
||||
|
@ -856,20 +916,20 @@ public final class CellUtil {
|
|||
|
||||
public static boolean matchingValue(final Cell left, final Cell right, int lvlength,
|
||||
int rvlength) {
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferedCell) right).getValuePosition(), rvlength);
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength,
|
||||
((ByteBufferCell) right).getValueByteBuffer(),
|
||||
((ByteBufferCell) right).getValuePosition(), rvlength);
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), lvlength, right.getValueArray(),
|
||||
right.getValueOffset(), rvlength);
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) right).getValueByteBuffer(),
|
||||
((ByteBufferedCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getValueByteBuffer(),
|
||||
((ByteBufferCell) right).getValuePosition(), rvlength, left.getValueArray(),
|
||||
left.getValueOffset(), lvlength);
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), lvlength,
|
||||
|
@ -877,9 +937,9 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
public static boolean matchingValue(final Cell left, final byte[] buf) {
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferedCell) left).getValueByteBuffer(),
|
||||
((ByteBufferedCell) left).getValuePosition(), left.getValueLength(), buf, 0,
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(((ByteBufferCell) left).getValueByteBuffer(),
|
||||
((ByteBufferCell) left).getValuePosition(), left.getValueLength(), buf, 0,
|
||||
buf.length) == 0;
|
||||
}
|
||||
return Bytes.equals(left.getValueArray(), left.getValueOffset(), left.getValueLength(), buf, 0,
|
||||
|
@ -1105,9 +1165,9 @@ public final class CellUtil {
|
|||
if (tagsLength == 0) {
|
||||
return TagUtil.EMPTY_TAGS_ITR;
|
||||
}
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return tagsIterator(((ByteBufferedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferedCell) cell).getTagsPosition(), tagsLength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), tagsLength);
|
||||
}
|
||||
return tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
|
@ -1133,14 +1193,14 @@ public final class CellUtil {
|
|||
* @return null if there is no tag of the passed in tag type
|
||||
*/
|
||||
public static Tag getTag(Cell cell, byte type){
|
||||
boolean bufferBacked = cell instanceof ByteBufferedCell;
|
||||
boolean bufferBacked = cell instanceof ByteBufferCell;
|
||||
int length = cell.getTagsLength();
|
||||
int offset = bufferBacked? ((ByteBufferedCell)cell).getTagsPosition():cell.getTagsOffset();
|
||||
int offset = bufferBacked? ((ByteBufferCell)cell).getTagsPosition():cell.getTagsOffset();
|
||||
int pos = offset;
|
||||
while (pos < offset + length) {
|
||||
int tagLen;
|
||||
if (bufferBacked) {
|
||||
ByteBuffer tagsBuffer = ((ByteBufferedCell)cell).getTagsByteBuffer();
|
||||
ByteBuffer tagsBuffer = ((ByteBufferCell)cell).getTagsByteBuffer();
|
||||
tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
|
||||
if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
|
||||
return new OffheapTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
|
||||
|
@ -1264,15 +1324,15 @@ public final class CellUtil {
|
|||
int qLen = cell.getQualifierLength();
|
||||
// Using just one if/else loop instead of every time checking before writing every
|
||||
// component of cell
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
out.writeShort(rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), rowLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), rowLen);
|
||||
out.writeByte(fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), qLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), fLen);
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), qLen);
|
||||
} else {
|
||||
out.writeShort(rowLen);
|
||||
out.write(cell.getRowArray(), cell.getRowOffset(), rowLen);
|
||||
|
@ -1292,9 +1352,9 @@ public final class CellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeRow(DataOutputStream out, Cell cell, short rlength) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), rlength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), rlength);
|
||||
} else {
|
||||
out.write(cell.getRowArray(), cell.getRowOffset(), rlength);
|
||||
}
|
||||
|
@ -1309,9 +1369,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static void writeRowSkippingBytes(DataOutputStream out, Cell cell, short rlength,
|
||||
int commonPrefix) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition() + commonPrefix, rlength - commonPrefix);
|
||||
} else {
|
||||
out.write(cell.getRowArray(), cell.getRowOffset() + commonPrefix, rlength - commonPrefix);
|
||||
}
|
||||
|
@ -1325,9 +1385,9 @@ public final class CellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeFamily(DataOutputStream out, Cell cell, byte flength) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), flength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), flength);
|
||||
} else {
|
||||
out.write(cell.getFamilyArray(), cell.getFamilyOffset(), flength);
|
||||
}
|
||||
|
@ -1342,9 +1402,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static void writeQualifier(DataOutputStream out, Cell cell, int qlength)
|
||||
throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), qlength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), qlength);
|
||||
} else {
|
||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset(), qlength);
|
||||
}
|
||||
|
@ -1359,9 +1419,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static void writeQualifierSkippingBytes(DataOutputStream out, Cell cell,
|
||||
int qlength, int commonPrefix) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition() + commonPrefix, qlength - commonPrefix);
|
||||
} else {
|
||||
out.write(cell.getQualifierArray(), cell.getQualifierOffset() + commonPrefix,
|
||||
qlength - commonPrefix);
|
||||
|
@ -1376,9 +1436,9 @@ public final class CellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeValue(DataOutputStream out, Cell cell, int vlength) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition(), vlength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), vlength);
|
||||
} else {
|
||||
out.write(cell.getValueArray(), cell.getValueOffset(), vlength);
|
||||
}
|
||||
|
@ -1392,9 +1452,9 @@ public final class CellUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static void writeTags(DataOutputStream out, Cell cell, int tagsLength) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferedCell) cell).getTagsPosition(), tagsLength);
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
ByteBufferUtils.copyBufferToStream(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), tagsLength);
|
||||
} else {
|
||||
out.write(cell.getTagsArray(), cell.getTagsOffset(), tagsLength);
|
||||
}
|
||||
|
@ -1494,10 +1554,10 @@ public final class CellUtil {
|
|||
}
|
||||
// Compare the RKs
|
||||
int rkCommonPrefix = 0;
|
||||
if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
|
||||
rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getRowByteBuffer(),
|
||||
((ByteBufferedCell) c1).getRowPosition(), rLen1, ((ByteBufferedCell) c2).getRowByteBuffer(),
|
||||
((ByteBufferedCell) c2).getRowPosition(), rLen2);
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
rkCommonPrefix = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getRowByteBuffer(),
|
||||
((ByteBufferCell) c1).getRowPosition(), rLen1, ((ByteBufferCell) c2).getRowByteBuffer(),
|
||||
((ByteBufferCell) c2).getRowPosition(), rLen2);
|
||||
} else {
|
||||
// There cannot be a case where one cell is BBCell and other is KeyValue. This flow comes either
|
||||
// in flush or compactions. In flushes both cells are KV and in case of compaction it will be either
|
||||
|
@ -1526,12 +1586,12 @@ public final class CellUtil {
|
|||
commonPrefix += KeyValue.FAMILY_LENGTH_SIZE;
|
||||
// Compare the CF names
|
||||
int fCommonPrefix;
|
||||
if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
fCommonPrefix =
|
||||
ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) c1).getFamilyPosition(), fLen1,
|
||||
((ByteBufferedCell) c2).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) c2).getFamilyPosition(), fLen2);
|
||||
ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) c1).getFamilyPosition(), fLen1,
|
||||
((ByteBufferCell) c2).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) c2).getFamilyPosition(), fLen2);
|
||||
} else {
|
||||
fCommonPrefix = ByteBufferUtils.findCommonPrefix(c1.getFamilyArray(), c1.getFamilyOffset(),
|
||||
fLen1, c2.getFamilyArray(), c2.getFamilyOffset(), fLen2);
|
||||
|
@ -1545,11 +1605,11 @@ public final class CellUtil {
|
|||
int qLen1 = c1.getQualifierLength();
|
||||
int qLen2 = c2.getQualifierLength();
|
||||
int qCommon;
|
||||
if (c1 instanceof ByteBufferedCell && c2 instanceof ByteBufferedCell) {
|
||||
qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferedCell) c1).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) c1).getQualifierPosition(), qLen1,
|
||||
((ByteBufferedCell) c2).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) c2).getQualifierPosition(), qLen2);
|
||||
if (c1 instanceof ByteBufferCell && c2 instanceof ByteBufferCell) {
|
||||
qCommon = ByteBufferUtils.findCommonPrefix(((ByteBufferCell) c1).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) c1).getQualifierPosition(), qLen1,
|
||||
((ByteBufferCell) c2).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) c2).getQualifierPosition(), qLen2);
|
||||
} else {
|
||||
qCommon = ByteBufferUtils.findCommonPrefix(c1.getQualifierArray(), c1.getQualifierOffset(),
|
||||
qLen1, c2.getQualifierArray(), c2.getQualifierOffset(), qLen2);
|
||||
|
@ -1658,20 +1718,20 @@ public final class CellUtil {
|
|||
short lrowlength = left.getRowLength();
|
||||
short rrowlength = right.getRowLength();
|
||||
if (lrowlength != rrowlength) return false;
|
||||
if (left instanceof ByteBufferedCell && right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), lrowlength,
|
||||
((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPosition(), rrowlength);
|
||||
if (left instanceof ByteBufferCell && right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), lrowlength,
|
||||
((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), rrowlength);
|
||||
}
|
||||
if (left instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) left).getRowByteBuffer(),
|
||||
((ByteBufferedCell) left).getRowPosition(), lrowlength, right.getRowArray(),
|
||||
if (left instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) left).getRowByteBuffer(),
|
||||
((ByteBufferCell) left).getRowPosition(), lrowlength, right.getRowArray(),
|
||||
right.getRowOffset(), rrowlength);
|
||||
}
|
||||
if (right instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferedCell) right).getRowByteBuffer(),
|
||||
((ByteBufferedCell) right).getRowPosition(), rrowlength, left.getRowArray(),
|
||||
if (right instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.equals(((ByteBufferCell) right).getRowByteBuffer(),
|
||||
((ByteBufferCell) right).getRowPosition(), rrowlength, left.getRowArray(),
|
||||
left.getRowOffset(), lrowlength);
|
||||
}
|
||||
return Bytes.equals(left.getRowArray(), left.getRowOffset(), lrowlength,
|
||||
|
@ -1704,9 +1764,9 @@ public final class CellUtil {
|
|||
* @return rowkey as int
|
||||
*/
|
||||
public static int getRowAsInt(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toInt(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition());
|
||||
}
|
||||
return Bytes.toInt(cell.getRowArray(), cell.getRowOffset());
|
||||
}
|
||||
|
@ -1718,9 +1778,9 @@ public final class CellUtil {
|
|||
* @return value as long
|
||||
*/
|
||||
public static long getValueAsLong(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toLong(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toLong(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition());
|
||||
}
|
||||
return Bytes.toLong(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
@ -1732,9 +1792,9 @@ public final class CellUtil {
|
|||
* @return value as double
|
||||
*/
|
||||
public static double getValueAsDouble(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toDouble(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toDouble(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition());
|
||||
}
|
||||
return Bytes.toDouble(cell.getValueArray(), cell.getValueOffset());
|
||||
}
|
||||
|
@ -1746,9 +1806,9 @@ public final class CellUtil {
|
|||
* @return value as BigDecimal
|
||||
*/
|
||||
public static BigDecimal getValueAsBigDecimal(Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return ByteBufferUtils.toBigDecimal(((ByteBufferedCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferedCell) cell).getValuePosition(), cell.getValueLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.toBigDecimal(((ByteBufferCell) cell).getValueByteBuffer(),
|
||||
((ByteBufferCell) cell).getValuePosition(), cell.getValueLength());
|
||||
}
|
||||
return Bytes.toBigDecimal(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
|
||||
}
|
||||
|
@ -1760,9 +1820,9 @@ public final class CellUtil {
|
|||
* @return First possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createFirstOnRow(final Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return new FirstOnRowByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
@ -1796,12 +1856,12 @@ public final class CellUtil {
|
|||
* @return First possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createFirstOnRowCol(final Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
HConstants.EMPTY_BYTE_BUFFER, 0, (byte) 0,
|
||||
((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
cell.getRowLength(), HConstants.EMPTY_BYTE_ARRAY, 0, (byte)0, cell.getQualifierArray(),
|
||||
|
@ -1829,11 +1889,11 @@ public final class CellUtil {
|
|||
* @return Last possible Cell on passed Cell's rk:cf and passed qualifier.
|
||||
*/
|
||||
public static Cell createFirstOnRowCol(final Cell cell, byte[] qArray, int qoffest, int qlength) {
|
||||
if(cell instanceof ByteBufferedCell) {
|
||||
return new FirstOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
if(cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
ByteBuffer.wrap(qArray), qoffest, qlength);
|
||||
}
|
||||
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
|
@ -1850,13 +1910,13 @@ public final class CellUtil {
|
|||
* @param ts
|
||||
*/
|
||||
public static Cell createFirstOnRowColTS(Cell cell, long ts) {
|
||||
if(cell instanceof ByteBufferedCell) {
|
||||
return new FirstOnRowColTSByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength(),
|
||||
if(cell instanceof ByteBufferCell) {
|
||||
return new FirstOnRowColTSByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength(),
|
||||
ts);
|
||||
}
|
||||
return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(),
|
||||
|
@ -1871,9 +1931,9 @@ public final class CellUtil {
|
|||
* @return Last possible Cell on passed Cell's row.
|
||||
*/
|
||||
public static Cell createLastOnRow(final Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return new LastOnRowByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new LastOnRowByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength());
|
||||
}
|
||||
return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
}
|
||||
|
@ -1891,13 +1951,13 @@ public final class CellUtil {
|
|||
* @return Last possible Cell on passed Cell's rk:cf:q.
|
||||
*/
|
||||
public static Cell createLastOnRowCol(final Cell cell) {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
return new LastOnRowColByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferedCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferedCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferedCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferedCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
return new LastOnRowColByteBufferCell(((ByteBufferCell) cell).getRowByteBuffer(),
|
||||
((ByteBufferCell) cell).getRowPosition(), cell.getRowLength(),
|
||||
((ByteBufferCell) cell).getFamilyByteBuffer(),
|
||||
((ByteBufferCell) cell).getFamilyPosition(), cell.getFamilyLength(),
|
||||
((ByteBufferCell) cell).getQualifierByteBuffer(),
|
||||
((ByteBufferCell) cell).getQualifierPosition(), cell.getQualifierLength());
|
||||
}
|
||||
return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
|
||||
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
|
||||
|
@ -1926,9 +1986,9 @@ public final class CellUtil {
|
|||
*/
|
||||
public static void compressTags(DataOutputStream out, Cell cell,
|
||||
TagCompressionContext tagCompressionContext) throws IOException {
|
||||
if (cell instanceof ByteBufferedCell) {
|
||||
tagCompressionContext.compressTags(out, ((ByteBufferedCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferedCell) cell).getTagsPosition(), cell.getTagsLength());
|
||||
if (cell instanceof ByteBufferCell) {
|
||||
tagCompressionContext.compressTags(out, ((ByteBufferCell) cell).getTagsByteBuffer(),
|
||||
((ByteBufferCell) cell).getTagsPosition(), cell.getTagsLength());
|
||||
} else {
|
||||
tagCompressionContext.compressTags(out, cell.getTagsArray(), cell.getTagsOffset(),
|
||||
cell.getTagsLength());
|
||||
|
@ -2032,7 +2092,7 @@ public final class CellUtil {
|
|||
* These cells are used in reseeks/seeks to improve the read performance.
|
||||
* They are not real cells that are returned back to the clients
|
||||
*/
|
||||
private static abstract class EmptyByteBufferedCell extends ByteBufferedCell
|
||||
private static abstract class EmptyByteBufferCell extends ByteBufferCell
|
||||
implements SettableSequenceId {
|
||||
|
||||
@Override
|
||||
|
@ -2210,12 +2270,12 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowByteBufferedCell extends EmptyByteBufferedCell {
|
||||
private static class FirstOnRowByteBufferCell extends EmptyByteBufferCell {
|
||||
private final ByteBuffer rowBuff;
|
||||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public FirstOnRowByteBufferedCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
public FirstOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
this.rowBuff = row;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
|
@ -2248,12 +2308,12 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class LastOnRowByteBufferedCell extends EmptyByteBufferedCell {
|
||||
private static class LastOnRowByteBufferCell extends EmptyByteBufferCell {
|
||||
private final ByteBuffer rowBuff;
|
||||
private final int roffset;
|
||||
private final short rlength;
|
||||
|
||||
public LastOnRowByteBufferedCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
public LastOnRowByteBufferCell(final ByteBuffer row, int roffset, short rlength) {
|
||||
this.rowBuff = row;
|
||||
this.roffset = roffset;
|
||||
this.rlength = rlength;
|
||||
|
@ -2286,7 +2346,7 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowColByteBufferedCell extends FirstOnRowByteBufferedCell {
|
||||
private static class FirstOnRowColByteBufferCell extends FirstOnRowByteBufferCell {
|
||||
private final ByteBuffer famBuff;
|
||||
private final int famOffset;
|
||||
private final byte famLength;
|
||||
|
@ -2294,7 +2354,7 @@ public final class CellUtil {
|
|||
private final int colOffset;
|
||||
private final int colLength;
|
||||
|
||||
public FirstOnRowColByteBufferedCell(final ByteBuffer row, int roffset, short rlength,
|
||||
public FirstOnRowColByteBufferCell(final ByteBuffer row, int roffset, short rlength,
|
||||
final ByteBuffer famBuff, final int famOffset, final byte famLength, final ByteBuffer col,
|
||||
final int colOffset, final int colLength) {
|
||||
super(row, roffset, rlength);
|
||||
|
@ -2406,11 +2466,11 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class FirstOnRowColTSByteBufferedCell extends FirstOnRowColByteBufferedCell {
|
||||
private static class FirstOnRowColTSByteBufferCell extends FirstOnRowColByteBufferCell {
|
||||
|
||||
private long ts;
|
||||
|
||||
public FirstOnRowColTSByteBufferedCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
public FirstOnRowColTSByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset, int qlength,
|
||||
long ts) {
|
||||
super(rBuffer, roffset, rlength, fBuffer, foffset, flength, qBuffer, qoffset, qlength);
|
||||
|
@ -2513,7 +2573,7 @@ public final class CellUtil {
|
|||
}
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static class LastOnRowColByteBufferedCell extends LastOnRowByteBufferedCell {
|
||||
private static class LastOnRowColByteBufferCell extends LastOnRowByteBufferCell {
|
||||
private final ByteBuffer fBuffer;
|
||||
private final int foffset;
|
||||
private final byte flength;
|
||||
|
@ -2521,7 +2581,7 @@ public final class CellUtil {
|
|||
private final int qoffset;
|
||||
private final int qlength;
|
||||
|
||||
public LastOnRowColByteBufferedCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
public LastOnRowColByteBufferCell(ByteBuffer rBuffer, int roffset, short rlength,
|
||||
ByteBuffer fBuffer, int foffset, byte flength, ByteBuffer qBuffer, int qoffset,
|
||||
int qlength) {
|
||||
super(rBuffer, roffset, rlength);
|
||||
|
@ -2604,4 +2664,34 @@ public final class CellUtil {
|
|||
return Type.DeleteFamily.getCode();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Clone the passed cell by copying its data into the passed buf.
|
||||
*/
|
||||
public static Cell copyCellTo(Cell cell, ByteBuffer buf, int offset, int len) {
|
||||
int tagsLen = cell.getTagsLength();
|
||||
if (cell instanceof ExtendedCell) {
|
||||
((ExtendedCell) cell).write(buf, offset);
|
||||
} else {
|
||||
// Normally all Cell impls within Server will be of type ExtendedCell. Just considering the
|
||||
// other case also. The data fragments within Cell is copied into buf as in KeyValue
|
||||
// serialization format only.
|
||||
KeyValueUtil.appendToByteBuffer(cell, buf, offset, true);
|
||||
}
|
||||
if (buf.hasArray()) {
|
||||
KeyValue newKv;
|
||||
if (tagsLen == 0) {
|
||||
// When tagsLen is 0, make a NoTagsKeyValue version of Cell. This is an optimized class
|
||||
// which directly return tagsLen as 0. So we avoid parsing many length components in
|
||||
// reading the tagLength stored in the backing buffer. The Memstore addition of every Cell
|
||||
// call getTagsLength().
|
||||
newKv = new NoTagsKeyValue(buf.array(), buf.arrayOffset() + offset, len);
|
||||
} else {
|
||||
newKv = new KeyValue(buf.array(), buf.arrayOffset() + offset, len);
|
||||
}
|
||||
newKv.setSequenceId(cell.getSequenceId());
|
||||
return newKv;
|
||||
}
|
||||
return new OffheapKeyValue(buf, offset, len, tagsLen > 0, cell.getSequenceId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
|
@ -65,7 +66,7 @@ public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestam
|
|||
* @param buf The buffer where to write the Cell.
|
||||
* @param offset The offset within buffer, to write the Cell.
|
||||
*/
|
||||
void write(byte[] buf, int offset);
|
||||
void write(ByteBuffer buf, int offset);
|
||||
|
||||
/**
|
||||
* @return The heap size overhead associated with this Cell.
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.util.VersionInfo;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKConfig;
|
||||
|
||||
|
@ -81,7 +80,6 @@ public class HBaseConfiguration extends Configuration {
|
|||
conf.addResource("hbase-site.xml");
|
||||
|
||||
checkDefaultsVersion(conf);
|
||||
HeapMemorySizeUtil.checkForClusterFreeMemoryLimit(conf);
|
||||
return conf;
|
||||
}
|
||||
|
||||
|
|
|
@ -2491,8 +2491,8 @@ public class KeyValue implements ExtendedCell {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] buf, int offset) {
|
||||
System.arraycopy(this.bytes, this.offset, buf, offset, this.length);
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, offset, this.bytes, this.offset, this.length);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -153,7 +153,6 @@ public class KeyValueUtil {
|
|||
return nextOffset;
|
||||
}
|
||||
|
||||
|
||||
/**************** copy key and value *********************/
|
||||
|
||||
public static int appendToByteArray(Cell cell, byte[] output, int offset, boolean withTags) {
|
||||
|
@ -170,15 +169,25 @@ public class KeyValueUtil {
|
|||
}
|
||||
|
||||
/**
|
||||
* The position will be set to the beginning of the new ByteBuffer
|
||||
* @param cell
|
||||
* @return the ByteBuffer containing the cell
|
||||
* Copy the Cell content into the passed buf in KeyValue serialization format.
|
||||
*/
|
||||
public static ByteBuffer copyToNewByteBuffer(final Cell cell) {
|
||||
byte[] bytes = new byte[length(cell)];
|
||||
appendToByteArray(cell, bytes, 0, true);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(bytes);
|
||||
return buffer;
|
||||
public static int appendToByteBuffer(Cell cell, ByteBuffer buf, int offset, boolean withTags) {
|
||||
offset = ByteBufferUtils.putInt(buf, offset, keyLength(cell));// Key length
|
||||
offset = ByteBufferUtils.putInt(buf, offset, cell.getValueLength());// Value length
|
||||
offset = ByteBufferUtils.putShort(buf, offset, cell.getRowLength());// RK length
|
||||
offset = CellUtil.copyRowTo(cell, buf, offset);// Row bytes
|
||||
offset = ByteBufferUtils.putByte(buf, offset, cell.getFamilyLength());// CF length
|
||||
offset = CellUtil.copyFamilyTo(cell, buf, offset);// CF bytes
|
||||
offset = CellUtil.copyQualifierTo(cell, buf, offset);// Qualifier bytes
|
||||
offset = ByteBufferUtils.putLong(buf, offset, cell.getTimestamp());// TS
|
||||
offset = ByteBufferUtils.putByte(buf, offset, cell.getTypeByte());// Type
|
||||
offset = CellUtil.copyValueTo(cell, buf, offset);// Value bytes
|
||||
int tagsLength = cell.getTagsLength();
|
||||
if (withTags && (tagsLength > 0)) {
|
||||
offset = ByteBufferUtils.putAsShort(buf, offset, tagsLength);// Tags length
|
||||
offset = CellUtil.copyTagTo(cell, buf, offset);// Tags bytes
|
||||
}
|
||||
return offset;
|
||||
}
|
||||
|
||||
public static void appendToByteBuffer(final ByteBuffer bb, final KeyValue kv,
|
||||
|
@ -660,29 +669,4 @@ public class KeyValueUtil {
|
|||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Write the given cell in KeyValue serialization format into the given buf and return a new
|
||||
* KeyValue object around that.
|
||||
*/
|
||||
public static KeyValue copyCellTo(Cell cell, byte[] buf, int offset, int len) {
|
||||
int tagsLen = cell.getTagsLength();
|
||||
if (cell instanceof ExtendedCell) {
|
||||
((ExtendedCell) cell).write(buf, offset);
|
||||
} else {
|
||||
appendToByteArray(cell, buf, offset, true);
|
||||
}
|
||||
KeyValue newKv;
|
||||
if (tagsLen == 0) {
|
||||
// When tagsLen is 0, make a NoTagsKeyValue version of Cell. This is an optimized class which
|
||||
// directly return tagsLen as 0. So we avoid parsing many length components in reading the
|
||||
// tagLength stored in the backing buffer. The Memstore addition of every Cell call
|
||||
// getTagsLength().
|
||||
newKv = new NoTagsKeyValue(buf, offset, len);
|
||||
} else {
|
||||
newKv = new KeyValue(buf, offset, len);
|
||||
}
|
||||
newKv.setSequenceId(cell.getSequenceId());
|
||||
return newKv;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,11 +27,11 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.ClassSize;
|
||||
|
||||
/**
|
||||
* This Cell is an implementation of {@link ByteBufferedCell} where the data resides in off heap
|
||||
* This Cell is an implementation of {@link ByteBufferCell} where the data resides in off heap
|
||||
* memory.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
|
||||
public class OffheapKeyValue extends ByteBufferCell implements ExtendedCell {
|
||||
|
||||
protected final ByteBuffer buf;
|
||||
protected final int offset;
|
||||
|
@ -266,8 +266,8 @@ public class OffheapKeyValue extends ByteBufferedCell implements ExtendedCell {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] buf, int offset) {
|
||||
ByteBufferUtils.copyFromBufferToArray(buf, this.buf, this.offset, offset, this.length);
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
ByteBufferUtils.copyFromBufferToBuffer(this.buf, buf, this.offset, offset, this.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.io.encoding;
|
|||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -55,7 +55,7 @@ public abstract class AbstractDataBlockEncoder implements DataBlockEncoder {
|
|||
return new KeyValue.KeyOnlyKeyValue(key.array(), key.arrayOffset()
|
||||
+ key.position(), keyLength);
|
||||
} else {
|
||||
return new ByteBufferedKeyOnlyKeyValue(key, key.position(), keyLength);
|
||||
return new ByteBufferKeyOnlyKeyValue(key, key.position(), keyLength);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,7 @@ import java.io.IOException;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -452,7 +452,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] buf, int offset) {
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
// This is not used in actual flow. Throwing UnsupportedOperationException
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
@ -475,7 +475,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
}
|
||||
}
|
||||
|
||||
protected static class OffheapDecodedCell extends ByteBufferedCell implements ExtendedCell {
|
||||
protected static class OffheapDecodedCell extends ByteBufferCell implements ExtendedCell {
|
||||
private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||
+ (3 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (7 * Bytes.SIZEOF_INT)
|
||||
+ (Bytes.SIZEOF_SHORT) + (2 * Bytes.SIZEOF_BYTE) + (3 * ClassSize.BYTE_BUFFER));
|
||||
|
@ -708,7 +708,7 @@ abstract class BufferedDataBlockEncoder extends AbstractDataBlockEncoder {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] buf, int offset) {
|
||||
public void write(ByteBuffer buf, int offset) {
|
||||
// This is not used in actual flow. Throwing UnsupportedOperationException
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -18,8 +18,8 @@ package org.apache.hadoop.hbase.io.encoding;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -153,10 +153,10 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
}
|
||||
|
||||
private int compareRows(ByteBuffer row, Cell seekCell) {
|
||||
if (seekCell instanceof ByteBufferedCell) {
|
||||
if (seekCell instanceof ByteBufferCell) {
|
||||
return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
|
||||
((ByteBufferedCell) seekCell).getRowByteBuffer(),
|
||||
((ByteBufferedCell) seekCell).getRowPosition(),
|
||||
((ByteBufferCell) seekCell).getRowByteBuffer(),
|
||||
((ByteBufferCell) seekCell).getRowPosition(),
|
||||
seekCell.getRowLength());
|
||||
} else {
|
||||
return ByteBufferUtils.compareTo(row, row.position(), row.remaining(),
|
||||
|
@ -315,7 +315,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
protected long memstoreTS;
|
||||
protected int nextKvOffset;
|
||||
// buffer backed keyonlyKV
|
||||
private ByteBufferedKeyOnlyKeyValue currentKey = new ByteBufferedKeyOnlyKeyValue();
|
||||
private ByteBufferKeyOnlyKeyValue currentKey = new ByteBufferKeyOnlyKeyValue();
|
||||
|
||||
protected boolean isValid() {
|
||||
return valueOffset != -1;
|
||||
|
@ -323,7 +323,7 @@ public class RowIndexSeekerV1 extends AbstractEncodedSeeker {
|
|||
|
||||
protected void invalidate() {
|
||||
valueOffset = -1;
|
||||
currentKey = new ByteBufferedKeyOnlyKeyValue();
|
||||
currentKey = new ByteBufferKeyOnlyKeyValue();
|
||||
currentBuffer = null;
|
||||
}
|
||||
|
||||
|
|
|
@ -876,6 +876,14 @@ public final class ByteBufferUtils {
|
|||
}
|
||||
}
|
||||
|
||||
public static int putInt(ByteBuffer buffer, int index, int val) {
|
||||
if (UNSAFE_UNALIGNED) {
|
||||
return UnsafeAccess.putInt(buffer, index, val);
|
||||
}
|
||||
buffer.putInt(index, val);
|
||||
return index + Bytes.SIZEOF_INT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads a double value at the given buffer's offset.
|
||||
* @param buffer
|
||||
|
@ -919,6 +927,21 @@ public final class ByteBufferUtils {
|
|||
}
|
||||
}
|
||||
|
||||
public static int putShort(ByteBuffer buffer, int index, short val) {
|
||||
if (UNSAFE_UNALIGNED) {
|
||||
return UnsafeAccess.putShort(buffer, index, val);
|
||||
}
|
||||
buffer.putShort(index, val);
|
||||
return index + Bytes.SIZEOF_SHORT;
|
||||
}
|
||||
|
||||
public static int putAsShort(ByteBuffer buf, int index, int val) {
|
||||
buf.put(index + 1, (byte) val);
|
||||
val >>= 8;
|
||||
buf.put(index, (byte) val);
|
||||
return index + Bytes.SIZEOF_SHORT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Put a long value out to the given ByteBuffer's current position in big-endian format.
|
||||
* This also advances the position in buffer by long size.
|
||||
|
@ -933,6 +956,15 @@ public final class ByteBufferUtils {
|
|||
buffer.putLong(val);
|
||||
}
|
||||
}
|
||||
|
||||
public static int putLong(ByteBuffer buffer, int index, long val) {
|
||||
if (UNSAFE_UNALIGNED) {
|
||||
return UnsafeAccess.putLong(buffer, index, val);
|
||||
}
|
||||
buffer.putLong(index, val);
|
||||
return index + Bytes.SIZEOF_LONG;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the bytes from given array's offset to length part into the given buffer. Puts the bytes
|
||||
* to buffer's current position. This also advances the position in the 'out' buffer by 'length'
|
||||
|
|
|
@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -102,15 +102,15 @@ public class TestCellComparator {
|
|||
byte[] v = Bytes.toBytes("val1");
|
||||
KeyValue kv = new KeyValue(r1, f1, q1, v);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
kv = new KeyValue(r2, f1, q1, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareColumns(bbCell1, bbCell2));
|
||||
assertEquals(0, CellComparator.compareColumns(bbCell1, kv));
|
||||
kv = new KeyValue(r2, f1, q2, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell3 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell3 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(0, CellComparator.compareFamilies(bbCell2, bbCell3));
|
||||
assertTrue(CellComparator.compareQualifiers(bbCell2, bbCell3) < 0);
|
||||
assertTrue(CellComparator.compareColumns(bbCell2, bbCell3) < 0);
|
||||
|
|
|
@ -416,7 +416,7 @@ public class TestCellUtil {
|
|||
byte[] tags = Bytes.toBytes("tag1");
|
||||
KeyValue kv = new KeyValue(r, f, q, 0, q.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
byte[] rDest = CellUtil.cloneRow(bbCell);
|
||||
assertTrue(Bytes.equals(r, rDest));
|
||||
byte[] fDest = CellUtil.cloneFamily(bbCell);
|
||||
|
@ -440,10 +440,10 @@ public class TestCellUtil {
|
|||
byte[] tags = Bytes.toBytes("tag1");
|
||||
KeyValue kv = new KeyValue(r, f, q1, 0, q1.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell1 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell1 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
kv = new KeyValue(r, f, q2, 0, q2.length, 1234L, Type.Put, v, 0, v.length, tags);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell2 = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell2 = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertTrue(CellUtil.matchingRows(bbCell1, bbCell2));
|
||||
assertTrue(CellUtil.matchingRows(kv, bbCell2));
|
||||
assertTrue(CellUtil.matchingRow(bbCell1, r));
|
||||
|
@ -473,30 +473,30 @@ public class TestCellUtil {
|
|||
byte[] v = Bytes.toBytes(vl);
|
||||
KeyValue kv = new KeyValue(r, f, q, v);
|
||||
ByteBuffer buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
Cell bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
Cell bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(ri, CellUtil.getRowAsInt(bbCell));
|
||||
assertEquals(vl, CellUtil.getValueAsLong(bbCell));
|
||||
double vd = 3005.5;
|
||||
v = Bytes.toBytes(vd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(vd, CellUtil.getValueAsDouble(bbCell), 0.0);
|
||||
BigDecimal bd = new BigDecimal(9999);
|
||||
v = Bytes.toBytes(bd);
|
||||
kv = new KeyValue(r, f, q, v);
|
||||
buffer = ByteBuffer.wrap(kv.getBuffer());
|
||||
bbCell = new ByteBufferedCellImpl(buffer, 0, buffer.remaining());
|
||||
bbCell = new ByteBufferCellImpl(buffer, 0, buffer.remaining());
|
||||
assertEquals(bd, CellUtil.getValueAsBigDecimal(bbCell));
|
||||
}
|
||||
|
||||
// TODO remove this test impl once we have a Cell implementation backed by ByteBuffer
|
||||
public static class ByteBufferedCellImpl extends ByteBufferedCell {
|
||||
public static class ByteBufferCellImpl extends ByteBufferCell {
|
||||
|
||||
private final ByteBuffer buffer;
|
||||
private final int offset, length;
|
||||
|
||||
public ByteBufferedCellImpl(ByteBuffer buffer, int offset, int length) {
|
||||
public ByteBufferCellImpl(ByteBuffer buffer, int offset, int length) {
|
||||
this.buffer = buffer;
|
||||
this.offset = offset;
|
||||
this.length = length;
|
||||
|
|
|
@ -56,7 +56,7 @@ public class TestOffheapKeyValue {
|
|||
KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
ByteBufferedCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
|
||||
ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), false, 0l);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
||||
|
@ -138,7 +138,7 @@ public class TestOffheapKeyValue {
|
|||
KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1, tags);
|
||||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
|
||||
ByteBufferedCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), true, 0l);
|
||||
ByteBufferCell offheapKV = new OffheapKeyValue(buf, 0, buf.capacity(), true, 0l);
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
|
||||
|
@ -178,7 +178,7 @@ public class TestOffheapKeyValue {
|
|||
ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getKeyLength());
|
||||
ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), kvCell.getKeyOffset(),
|
||||
kvCell.getKeyLength());
|
||||
ByteBufferedCell offheapKeyOnlyKV = new ByteBufferedKeyOnlyKeyValue(buf, 0, buf.capacity());
|
||||
ByteBufferCell offheapKeyOnlyKV = new ByteBufferKeyOnlyKeyValue(buf, 0, buf.capacity());
|
||||
assertEquals(
|
||||
ROW1,
|
||||
ByteBufferUtils.toStringBinary(offheapKeyOnlyKV.getRowByteBuffer(),
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.OffheapKeyValue;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.io.util.LRUDictionary;
|
||||
import org.apache.hadoop.hbase.nio.SingleByteBuff;
|
||||
|
@ -80,10 +80,10 @@ public class TestTagCompressionContext {
|
|||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
|
||||
TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
|
||||
ByteBufferedCell kv1 = (ByteBufferedCell)createOffheapKVWithTags(2);
|
||||
ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(2);
|
||||
int tagsLength1 = kv1.getTagsLength();
|
||||
context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
|
||||
ByteBufferedCell kv2 = (ByteBufferedCell)createOffheapKVWithTags(3);
|
||||
ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
|
||||
int tagsLength2 = kv2.getTagsLength();
|
||||
context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
|
||||
|
||||
|
@ -129,10 +129,10 @@ public class TestTagCompressionContext {
|
|||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
DataOutputStream daos = new ByteBufferWriterDataOutputStream(baos);
|
||||
TagCompressionContext context = new TagCompressionContext(LRUDictionary.class, Byte.MAX_VALUE);
|
||||
ByteBufferedCell kv1 = (ByteBufferedCell)createOffheapKVWithTags(1);
|
||||
ByteBufferCell kv1 = (ByteBufferCell)createOffheapKVWithTags(1);
|
||||
int tagsLength1 = kv1.getTagsLength();
|
||||
context.compressTags(daos, kv1.getTagsByteBuffer(), kv1.getTagsPosition(), tagsLength1);
|
||||
ByteBufferedCell kv2 = (ByteBufferedCell)createOffheapKVWithTags(3);
|
||||
ByteBufferCell kv2 = (ByteBufferCell)createOffheapKVWithTags(3);
|
||||
int tagsLength2 = kv2.getTagsLength();
|
||||
context.compressTags(daos, kv2.getTagsByteBuffer(), kv2.getTagsPosition(), tagsLength2);
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.prefixtree;
|
|||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -93,7 +93,7 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
// The PrefixTreecell is of type BytebufferedCell and the value part of the cell
|
||||
// determines whether we are offheap cell or onheap cell. All other parts of the cell-
|
||||
// row, fam and col are all represented as onheap byte[]
|
||||
ByteBufferedCell cell = (ByteBufferedCell)ptSearcher.current();
|
||||
ByteBufferCell cell = (ByteBufferCell)ptSearcher.current();
|
||||
if (cell == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -373,7 +373,7 @@ public class PrefixTreeSeeker implements EncodedSeeker {
|
|||
}
|
||||
}
|
||||
|
||||
private static class OffheapPrefixTreeCell extends ByteBufferedCell implements Cell,
|
||||
private static class OffheapPrefixTreeCell extends ByteBufferCell implements Cell,
|
||||
SettableSequenceId, HeapSize {
|
||||
private static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||
+ (5 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_INT)
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.decode;
|
|||
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.util.ObjectIntPair;
|
|||
* without allocating new memory for every Cell iterated through.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class PrefixTreeCell extends ByteBufferedCell implements SettableSequenceId,
|
||||
public class PrefixTreeCell extends ByteBufferCell implements SettableSequenceId,
|
||||
Comparable<Cell> {
|
||||
// Create a reference here? Can be removed too
|
||||
protected CellComparator comparator = CellComparator.COMPARATOR;
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -745,7 +745,7 @@ public class HFileBlockIndex {
|
|||
// If we imagine that keys[-1] = -Infinity and
|
||||
// keys[numEntries] = Infinity, then we are maintaining an invariant that
|
||||
// keys[low - 1] < key < keys[high + 1] while narrowing down the range.
|
||||
ByteBufferedKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferedKeyOnlyKeyValue();
|
||||
ByteBufferKeyOnlyKeyValue nonRootIndexkeyOnlyKV = new ByteBufferKeyOnlyKeyValue();
|
||||
ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
|
||||
while (low <= high) {
|
||||
mid = (low + high) >>> 1;
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferedKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -491,7 +491,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
protected final HFile.Reader reader;
|
||||
private int currTagsLen;
|
||||
// buffer backed keyonlyKV
|
||||
private ByteBufferedKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferedKeyOnlyKeyValue();
|
||||
private ByteBufferKeyOnlyKeyValue bufBackedKeyOnlyKv = new ByteBufferKeyOnlyKeyValue();
|
||||
// A pair for reusing in blockSeek() so that we don't garbage lot of objects
|
||||
final ObjectIntPair<ByteBuffer> pair = new ObjectIntPair<ByteBuffer>();
|
||||
|
||||
|
@ -1208,7 +1208,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
|
|||
return new KeyValue.KeyOnlyKeyValue(keyBuff.array(), keyBuff.arrayOffset()
|
||||
+ keyBuff.position(), klen);
|
||||
} else {
|
||||
return new ByteBufferedKeyOnlyKeyValue(keyBuff, keyBuff.position(), klen);
|
||||
return new ByteBufferKeyOnlyKeyValue(keyBuff, keyBuff.position(), klen);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hbase.io.util;
|
||||
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.MemoryType;
|
||||
import java.lang.management.MemoryUsage;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -25,9 +26,14 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStoreLAB;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
/**
|
||||
* Util class to calculate memory size for memstore, block cache(L1, L2) of RS.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HeapMemorySizeUtil {
|
||||
public class MemorySizeUtil {
|
||||
|
||||
public static final String MEMSTORE_SIZE_KEY = "hbase.regionserver.global.memstore.size";
|
||||
public static final String MEMSTORE_SIZE_OLD_KEY =
|
||||
|
@ -36,12 +42,16 @@ public class HeapMemorySizeUtil {
|
|||
"hbase.regionserver.global.memstore.size.lower.limit";
|
||||
public static final String MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY =
|
||||
"hbase.regionserver.global.memstore.lowerLimit";
|
||||
// Max global off heap memory that can be used for all memstores
|
||||
// This should be an absolute value in MBs and not percent.
|
||||
public static final String OFFHEAP_MEMSTORE_SIZE_KEY =
|
||||
"hbase.regionserver.offheap.global.memstore.size";
|
||||
|
||||
public static final float DEFAULT_MEMSTORE_SIZE = 0.4f;
|
||||
// Default lower water mark limit is 95% size of memstore size.
|
||||
public static final float DEFAULT_MEMSTORE_SIZE_LOWER_LIMIT = 0.95f;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(HeapMemorySizeUtil.class);
|
||||
private static final Log LOG = LogFactory.getLog(MemorySizeUtil.class);
|
||||
// a constant to convert a fraction to a percentage
|
||||
private static final int CONVERT_TO_PERCENTAGE = 100;
|
||||
|
||||
|
@ -50,11 +60,11 @@ public class HeapMemorySizeUtil {
|
|||
* We need atleast 20% of heap left out for other RS functions.
|
||||
* @param conf
|
||||
*/
|
||||
public static void checkForClusterFreeMemoryLimit(Configuration conf) {
|
||||
public static void checkForClusterFreeHeapMemoryLimit(Configuration conf) {
|
||||
if (conf.get(MEMSTORE_SIZE_OLD_KEY) != null) {
|
||||
LOG.warn(MEMSTORE_SIZE_OLD_KEY + " is deprecated by " + MEMSTORE_SIZE_KEY);
|
||||
}
|
||||
float globalMemstoreSize = getGlobalMemStorePercent(conf, false);
|
||||
float globalMemstoreSize = getGlobalMemStoreHeapPercent(conf, false);
|
||||
int gml = (int)(globalMemstoreSize * CONVERT_TO_PERCENTAGE);
|
||||
float blockCacheUpperLimit = getBlockCacheHeapPercent(conf);
|
||||
int bcul = (int)(blockCacheUpperLimit * CONVERT_TO_PERCENTAGE);
|
||||
|
@ -76,7 +86,8 @@ public class HeapMemorySizeUtil {
|
|||
* @param c
|
||||
* @param logInvalid
|
||||
*/
|
||||
public static float getGlobalMemStorePercent(final Configuration c, final boolean logInvalid) {
|
||||
public static float getGlobalMemStoreHeapPercent(final Configuration c,
|
||||
final boolean logInvalid) {
|
||||
float limit = c.getFloat(MEMSTORE_SIZE_KEY,
|
||||
c.getFloat(MEMSTORE_SIZE_OLD_KEY, DEFAULT_MEMSTORE_SIZE));
|
||||
if (limit > 0.8f || limit <= 0.0f) {
|
||||
|
@ -93,34 +104,64 @@ public class HeapMemorySizeUtil {
|
|||
* Retrieve configured size for global memstore lower water mark as fraction of global memstore
|
||||
* size.
|
||||
*/
|
||||
public static float getGlobalMemStoreLowerMark(final Configuration conf, float globalMemStorePercent) {
|
||||
public static float getGlobalMemStoreHeapLowerMark(final Configuration conf,
|
||||
boolean honorOldConfig) {
|
||||
String lowMarkPercentStr = conf.get(MEMSTORE_SIZE_LOWER_LIMIT_KEY);
|
||||
if (lowMarkPercentStr != null) {
|
||||
float lowMarkPercent = Float.parseFloat(lowMarkPercentStr);
|
||||
if (lowMarkPercent > 1.0f) {
|
||||
LOG.error("Bad configuration value for " + MEMSTORE_SIZE_LOWER_LIMIT_KEY + ": " +
|
||||
lowMarkPercent + ". Using 1.0f instead.");
|
||||
LOG.error("Bad configuration value for " + MEMSTORE_SIZE_LOWER_LIMIT_KEY + ": "
|
||||
+ lowMarkPercent + ". Using 1.0f instead.");
|
||||
lowMarkPercent = 1.0f;
|
||||
}
|
||||
return lowMarkPercent;
|
||||
}
|
||||
if (!honorOldConfig) return DEFAULT_MEMSTORE_SIZE_LOWER_LIMIT;
|
||||
String lowerWaterMarkOldValStr = conf.get(MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY);
|
||||
if (lowerWaterMarkOldValStr != null) {
|
||||
LOG.warn(MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY + " is deprecated. Instead use "
|
||||
+ MEMSTORE_SIZE_LOWER_LIMIT_KEY);
|
||||
float lowerWaterMarkOldVal = Float.parseFloat(lowerWaterMarkOldValStr);
|
||||
if (lowerWaterMarkOldVal > globalMemStorePercent) {
|
||||
lowerWaterMarkOldVal = globalMemStorePercent;
|
||||
float upperMarkPercent = getGlobalMemStoreHeapPercent(conf, false);
|
||||
if (lowerWaterMarkOldVal > upperMarkPercent) {
|
||||
lowerWaterMarkOldVal = upperMarkPercent;
|
||||
LOG.error("Value of " + MEMSTORE_SIZE_LOWER_LIMIT_OLD_KEY + " (" + lowerWaterMarkOldVal
|
||||
+ ") is greater than global memstore limit (" + globalMemStorePercent + ") set by "
|
||||
+ ") is greater than global memstore limit (" + upperMarkPercent + ") set by "
|
||||
+ MEMSTORE_SIZE_KEY + "/" + MEMSTORE_SIZE_OLD_KEY + ". Setting memstore lower limit "
|
||||
+ "to " + globalMemStorePercent);
|
||||
+ "to " + upperMarkPercent);
|
||||
}
|
||||
return lowerWaterMarkOldVal / globalMemStorePercent;
|
||||
return lowerWaterMarkOldVal / upperMarkPercent;
|
||||
}
|
||||
return DEFAULT_MEMSTORE_SIZE_LOWER_LIMIT;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Pair of global memstore size and memory type(ie. on heap or off heap).
|
||||
*/
|
||||
public static Pair<Long, MemoryType> getGlobalMemstoreSize(Configuration conf) {
|
||||
long offheapMSGlobal = conf.getLong(OFFHEAP_MEMSTORE_SIZE_KEY, 0);// Size in MBs
|
||||
if (offheapMSGlobal > 0) {
|
||||
// Off heap memstore size has not relevance when MSLAB is turned OFF. We will go with making
|
||||
// this entire size split into Chunks and pooling them in MemstoreLABPoool. We dont want to
|
||||
// create so many on demand off heap chunks. In fact when this off heap size is configured, we
|
||||
// will go with 100% of this size as the pool size
|
||||
if (MemStoreLAB.isEnabled(conf)) {
|
||||
// We are in offheap Memstore use
|
||||
long globalMemStoreLimit = (long) (offheapMSGlobal * 1024 * 1024); // Size in bytes
|
||||
return new Pair<Long, MemoryType>(globalMemStoreLimit, MemoryType.NON_HEAP);
|
||||
} else {
|
||||
// Off heap max memstore size is configured with turning off MSLAB. It makes no sense. Do a
|
||||
// warn log and go with on heap memstore percentage. By default it will be 40% of Xmx
|
||||
LOG.warn("There is no relevance of configuring '" + OFFHEAP_MEMSTORE_SIZE_KEY + "' when '"
|
||||
+ MemStoreLAB.USEMSLAB_KEY + "' is turned off."
|
||||
+ " Going with on heap global memstore size ('" + MEMSTORE_SIZE_KEY + "')");
|
||||
}
|
||||
}
|
||||
long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
|
||||
float globalMemStorePercent = getGlobalMemStoreHeapPercent(conf, true);
|
||||
return new Pair<Long, MemoryType>((long) (max * globalMemStorePercent), MemoryType.HEAP);
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve configured size for on heap block cache as percentage of total heap.
|
||||
* @param conf
|
|
@ -17,34 +17,34 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A chunk of memory out of which allocations are sliced.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class Chunk {
|
||||
public abstract class Chunk {
|
||||
/** Actual underlying data */
|
||||
private byte[] data;
|
||||
protected ByteBuffer data;
|
||||
|
||||
private static final int UNINITIALIZED = -1;
|
||||
private static final int OOM = -2;
|
||||
protected static final int UNINITIALIZED = -1;
|
||||
protected static final int OOM = -2;
|
||||
/**
|
||||
* Offset for the next allocation, or the sentinel value -1 which implies that the chunk is still
|
||||
* uninitialized.
|
||||
*/
|
||||
private AtomicInteger nextFreeOffset = new AtomicInteger(UNINITIALIZED);
|
||||
protected AtomicInteger nextFreeOffset = new AtomicInteger(UNINITIALIZED);
|
||||
|
||||
/** Total number of allocations satisfied from this buffer */
|
||||
private AtomicInteger allocCount = new AtomicInteger();
|
||||
protected AtomicInteger allocCount = new AtomicInteger();
|
||||
|
||||
/** Size of chunk in bytes */
|
||||
private final int size;
|
||||
protected final int size;
|
||||
|
||||
/**
|
||||
* Create an uninitialized chunk. Note that memory is not allocated yet, so this is cheap.
|
||||
|
@ -60,23 +60,7 @@ public class Chunk {
|
|||
* constructed the chunk. It is thread-safe against other threads calling alloc(), who will block
|
||||
* until the allocation is complete.
|
||||
*/
|
||||
public void init() {
|
||||
assert nextFreeOffset.get() == UNINITIALIZED;
|
||||
try {
|
||||
if (data == null) {
|
||||
data = new byte[size];
|
||||
}
|
||||
} catch (OutOfMemoryError e) {
|
||||
boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
|
||||
assert failInit; // should be true.
|
||||
throw e;
|
||||
}
|
||||
// Mark that it's ready for use
|
||||
boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
|
||||
// We should always succeed the above CAS since only one thread
|
||||
// calls init()!
|
||||
Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
|
||||
}
|
||||
public abstract void init();
|
||||
|
||||
/**
|
||||
* Reset the offset to UNINITIALIZED before before reusing an old chunk
|
||||
|
@ -109,7 +93,7 @@ public class Chunk {
|
|||
return -1;
|
||||
}
|
||||
|
||||
if (oldOffset + size > data.length) {
|
||||
if (oldOffset + size > data.capacity()) {
|
||||
return -1; // alloc doesn't fit
|
||||
}
|
||||
|
||||
|
@ -126,14 +110,14 @@ public class Chunk {
|
|||
/**
|
||||
* @return This chunk's backing data.
|
||||
*/
|
||||
byte[] getData() {
|
||||
ByteBuffer getData() {
|
||||
return this.data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "Chunk@" + System.identityHashCode(this) + " allocs=" + allocCount.get() + "waste="
|
||||
+ (data.length - nextFreeOffset.get());
|
||||
+ (data.capacity() - nextFreeOffset.get());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
|
||||
import org.apache.hadoop.hbase.util.RollingStatCalculator;
|
||||
|
@ -109,6 +109,9 @@ class DefaultHeapMemoryTuner implements HeapMemoryTuner {
|
|||
private float globalMemStorePercentMaxRange;
|
||||
private float blockCachePercentMinRange;
|
||||
private float blockCachePercentMaxRange;
|
||||
|
||||
private float globalMemStoreLimitLowMarkPercent;
|
||||
|
||||
// Store statistics about the corresponding parameters for memory tuning
|
||||
private RollingStatCalculator rollingStatsForCacheMisses;
|
||||
private RollingStatCalculator rollingStatsForFlushes;
|
||||
|
@ -165,11 +168,9 @@ class DefaultHeapMemoryTuner implements HeapMemoryTuner {
|
|||
newTuneDirection = StepDirection.NEUTRAL;
|
||||
}
|
||||
// Increase / decrease the memstore / block cahce sizes depending on new tuner step.
|
||||
float globalMemstoreLowerMark = HeapMemorySizeUtil.getGlobalMemStoreLowerMark(conf,
|
||||
curMemstoreSize);
|
||||
// We don't want to exert immediate pressure on memstore. So, we decrease its size gracefully;
|
||||
// we set a minimum bar in the middle of the total memstore size and the lower limit.
|
||||
float minMemstoreSize = ((globalMemstoreLowerMark + 1) * curMemstoreSize) / 2.00f;
|
||||
float minMemstoreSize = ((globalMemStoreLimitLowMarkPercent + 1) * curMemstoreSize) / 2.00f;
|
||||
|
||||
switch (newTuneDirection) {
|
||||
case INCREASE_BLOCK_CACHE_SIZE:
|
||||
|
@ -365,9 +366,11 @@ class DefaultHeapMemoryTuner implements HeapMemoryTuner {
|
|||
this.blockCachePercentMaxRange = conf.getFloat(BLOCK_CACHE_SIZE_MAX_RANGE_KEY,
|
||||
conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY, HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT));
|
||||
this.globalMemStorePercentMinRange = conf.getFloat(MEMSTORE_SIZE_MIN_RANGE_KEY,
|
||||
HeapMemorySizeUtil.getGlobalMemStorePercent(conf, false));
|
||||
MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
|
||||
this.globalMemStorePercentMaxRange = conf.getFloat(MEMSTORE_SIZE_MAX_RANGE_KEY,
|
||||
HeapMemorySizeUtil.getGlobalMemStorePercent(conf, false));
|
||||
MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
|
||||
this.globalMemStoreLimitLowMarkPercent = MemorySizeUtil.getGlobalMemStoreHeapLowerMark(conf,
|
||||
true);
|
||||
// Default value of periods to ignore is number of lookup periods
|
||||
this.numPeriodsToIgnore = conf.getInt(NUM_PERIODS_TO_IGNORE, this.tunerLookupPeriods);
|
||||
this.rollingStatsForCacheMisses = new RollingStatCalculator(this.tunerLookupPeriods);
|
||||
|
|
|
@ -7028,7 +7028,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
|
||||
stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreDataSize.get() * 100) / this
|
||||
.memstoreFlushSize)));
|
||||
stats.setHeapOccupancy((int)rsServices.getHeapMemoryManager().getHeapOccupancyPercent()*100);
|
||||
if (rsServices.getHeapMemoryManager() != null) {
|
||||
stats.setHeapOccupancy(
|
||||
(int) rsServices.getHeapMemoryManager().getHeapOccupancyPercent() * 100);
|
||||
}
|
||||
stats.setCompactionPressure((int)rsServices.getCompactionPressure()*100 > 100 ? 100 :
|
||||
(int)rsServices.getCompactionPressure()*100);
|
||||
return stats.build();
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import java.io.InterruptedIOException;
|
||||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.MemoryType;
|
||||
import java.lang.management.MemoryUsage;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.net.BindException;
|
||||
|
@ -100,6 +101,7 @@ import org.apache.hadoop.hbase.fs.HFileSystem;
|
|||
import org.apache.hadoop.hbase.http.InfoServer;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
|
||||
|
@ -170,6 +172,7 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
|||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.JSONBean;
|
||||
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.util.Sleeper;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
@ -516,6 +519,7 @@ public class HRegionServer extends HasThread implements
|
|||
super("RegionServer"); // thread name
|
||||
this.fsOk = true;
|
||||
this.conf = conf;
|
||||
MemorySizeUtil.checkForClusterFreeHeapMemoryLimit(this.conf);
|
||||
HFile.checkHFileVersion(this.conf);
|
||||
checkCodecs(this.conf);
|
||||
this.userProvider = UserProvider.instantiate(conf);
|
||||
|
@ -1451,6 +1455,8 @@ public class HRegionServer extends HasThread implements
|
|||
|
||||
startServiceThreads();
|
||||
startHeapMemoryManager();
|
||||
// Call it after starting HeapMemoryManager.
|
||||
initializeMemStoreChunkPool();
|
||||
LOG.info("Serving as " + this.serverName +
|
||||
", RpcServer on " + rpcServices.isa +
|
||||
", sessionid=0x" +
|
||||
|
@ -1470,16 +1476,34 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
}
|
||||
|
||||
private void initializeMemStoreChunkPool() {
|
||||
if (MemStoreLAB.isEnabled(conf)) {
|
||||
// MSLAB is enabled. So initialize MemStoreChunkPool
|
||||
// By this time, the MemstoreFlusher is already initialized. We can get the global limits from
|
||||
// it.
|
||||
Pair<Long, MemoryType> pair = MemorySizeUtil.getGlobalMemstoreSize(conf);
|
||||
long globalMemStoreSize = pair.getFirst();
|
||||
boolean offheap = pair.getSecond() == MemoryType.NON_HEAP;
|
||||
// When off heap memstore in use, take full area for chunk pool.
|
||||
float poolSizePercentage = offheap ? 1.0F
|
||||
: conf.getFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, MemStoreLAB.POOL_MAX_SIZE_DEFAULT);
|
||||
float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
|
||||
MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
|
||||
int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
|
||||
MemStoreChunkPool pool = MemStoreChunkPool.initialize(globalMemStoreSize, poolSizePercentage,
|
||||
initialCountPercentage, chunkSize, offheap);
|
||||
if (pool != null && this.hMemManager != null) {
|
||||
// Register with Heap Memory manager
|
||||
this.hMemManager.registerTuneObserver(pool);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void startHeapMemoryManager() {
|
||||
this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher,
|
||||
this, this.regionServerAccounting);
|
||||
this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this,
|
||||
this.regionServerAccounting);
|
||||
if (this.hMemManager != null) {
|
||||
this.hMemManager.start(getChoreService());
|
||||
MemStoreChunkPool chunkPool = MemStoreChunkPool.getPool(this.conf);
|
||||
if (chunkPool != null) {
|
||||
// Register it as HeapMemoryTuneObserver
|
||||
this.hMemManager.registerTuneObserver(chunkPool);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -3522,11 +3546,6 @@ public class HRegionServer extends HasThread implements
|
|||
configurationManager.notifyAllObservers(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HeapMemoryManager getHeapMemoryManager() {
|
||||
return hMemManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getCompactionPressure() {
|
||||
double max = 0;
|
||||
|
@ -3541,6 +3560,11 @@ public class HRegionServer extends HasThread implements
|
|||
return max;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HeapMemoryManager getHeapMemoryManager() {
|
||||
return hMemManager;
|
||||
}
|
||||
|
||||
/**
|
||||
* For testing
|
||||
* @return whether all wal roll request finished for this regionserver
|
||||
|
|
|
@ -36,13 +36,15 @@ import org.apache.hadoop.hbase.Server;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Manages tuning of Heap memory using <code>HeapMemoryTuner</code>.
|
||||
* Manages tuning of Heap memory using <code>HeapMemoryTuner</code>. Most part of the heap memory is
|
||||
* split between Memstores and BlockCache. This manager helps in tuning sizes of both these
|
||||
* dynamically, as per the R/W load on the servers.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HeapMemoryManager {
|
||||
|
@ -91,7 +93,7 @@ public class HeapMemoryManager {
|
|||
private List<HeapMemoryTuneObserver> tuneObservers = new ArrayList<HeapMemoryTuneObserver>();
|
||||
|
||||
public static HeapMemoryManager create(Configuration conf, FlushRequester memStoreFlusher,
|
||||
Server server, RegionServerAccounting regionServerAccounting) {
|
||||
Server server, RegionServerAccounting regionServerAccounting) {
|
||||
ResizableBlockCache l1Cache = CacheConfig.getL1(conf);
|
||||
if (l1Cache != null) {
|
||||
return new HeapMemoryManager(l1Cache, memStoreFlusher, server, regionServerAccounting);
|
||||
|
@ -117,10 +119,10 @@ public class HeapMemoryManager {
|
|||
|
||||
private boolean doInit(Configuration conf) {
|
||||
boolean tuningEnabled = true;
|
||||
globalMemStorePercent = HeapMemorySizeUtil.getGlobalMemStorePercent(conf, false);
|
||||
globalMemStorePercent = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false);
|
||||
blockCachePercent = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY,
|
||||
HConstants.HFILE_BLOCK_CACHE_SIZE_DEFAULT);
|
||||
HeapMemorySizeUtil.checkForClusterFreeMemoryLimit(conf);
|
||||
MemorySizeUtil.checkForClusterFreeHeapMemoryLimit(conf);
|
||||
// Initialize max and min range for memstore heap space
|
||||
globalMemStorePercentMinRange = conf.getFloat(MEMSTORE_SIZE_MIN_RANGE_KEY,
|
||||
globalMemStorePercent);
|
||||
|
@ -128,14 +130,14 @@ public class HeapMemoryManager {
|
|||
globalMemStorePercent);
|
||||
if (globalMemStorePercent < globalMemStorePercentMinRange) {
|
||||
LOG.warn("Setting " + MEMSTORE_SIZE_MIN_RANGE_KEY + " to " + globalMemStorePercent
|
||||
+ ", same value as " + HeapMemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ " because supplied value greater than initial memstore size value.");
|
||||
globalMemStorePercentMinRange = globalMemStorePercent;
|
||||
conf.setFloat(MEMSTORE_SIZE_MIN_RANGE_KEY, globalMemStorePercentMinRange);
|
||||
}
|
||||
if (globalMemStorePercent > globalMemStorePercentMaxRange) {
|
||||
LOG.warn("Setting " + MEMSTORE_SIZE_MAX_RANGE_KEY + " to " + globalMemStorePercent
|
||||
+ ", same value as " + HeapMemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ ", same value as " + MemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ " because supplied value less than initial memstore size value.");
|
||||
globalMemStorePercentMaxRange = globalMemStorePercent;
|
||||
conf.setFloat(MEMSTORE_SIZE_MAX_RANGE_KEY, globalMemStorePercentMaxRange);
|
||||
|
@ -167,7 +169,7 @@ public class HeapMemoryManager {
|
|||
}
|
||||
|
||||
int gml = (int) (globalMemStorePercentMaxRange * CONVERT_TO_PERCENTAGE);
|
||||
this.l2BlockCachePercent = HeapMemorySizeUtil.getL2BlockCacheHeapPercent(conf);
|
||||
this.l2BlockCachePercent = MemorySizeUtil.getL2BlockCacheHeapPercent(conf);
|
||||
int bcul = (int) ((blockCachePercentMinRange + l2BlockCachePercent) * CONVERT_TO_PERCENTAGE);
|
||||
if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
|
||||
throw new RuntimeException("Current heap configuration for MemStore and BlockCache exceeds "
|
||||
|
@ -340,7 +342,7 @@ public class HeapMemoryManager {
|
|||
if (CONVERT_TO_PERCENTAGE - (gml + bcul) < CLUSTER_MINIMUM_MEMORY_THRESHOLD) {
|
||||
LOG.info("Current heap configuration from HeapMemoryTuner exceeds "
|
||||
+ "the threshold required for successful cluster operation. "
|
||||
+ "The combined value cannot exceed 0.8. " + HeapMemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ "The combined value cannot exceed 0.8. " + MemorySizeUtil.MEMSTORE_SIZE_KEY
|
||||
+ " is " + memstoreSize + " and " + HFILE_BLOCK_CACHE_SIZE_KEY + " is "
|
||||
+ blockCacheSize);
|
||||
// TODO can adjust the value so as not exceed 80%. Is that correct? may be.
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
@ -29,8 +28,6 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.HeapMemoryTuneObserver;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
|
@ -45,7 +42,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
* collection on JVM.
|
||||
*
|
||||
* The pool instance is globally unique and could be obtained through
|
||||
* {@link MemStoreChunkPool#getPool(Configuration)}
|
||||
* {@link MemStoreChunkPool#initialize(long, float, float, int, boolean)}
|
||||
*
|
||||
* {@link MemStoreChunkPool#getChunk()} is called when MemStoreLAB allocating
|
||||
* bytes, and {@link MemStoreChunkPool#putbackChunks(BlockingQueue)} is called
|
||||
|
@ -55,10 +52,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|||
@InterfaceAudience.Private
|
||||
public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
||||
private static final Log LOG = LogFactory.getLog(MemStoreChunkPool.class);
|
||||
final static String CHUNK_POOL_MAXSIZE_KEY = "hbase.hregion.memstore.chunkpool.maxsize";
|
||||
final static String CHUNK_POOL_INITIALSIZE_KEY = "hbase.hregion.memstore.chunkpool.initialsize";
|
||||
final static float POOL_MAX_SIZE_DEFAULT = 1.0f;
|
||||
final static float POOL_INITIAL_SIZE_DEFAULT = 0.0f;
|
||||
|
||||
// Static reference to the MemStoreChunkPool
|
||||
static MemStoreChunkPool GLOBAL_INSTANCE;
|
||||
|
@ -68,7 +61,7 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
private int maxCount;
|
||||
|
||||
// A queue of reclaimed chunks
|
||||
private final BlockingQueue<PooledChunk> reclaimedChunks;
|
||||
private final BlockingQueue<Chunk> reclaimedChunks;
|
||||
private final int chunkSize;
|
||||
private final float poolSizePercentage;
|
||||
|
||||
|
@ -78,15 +71,17 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
private static final int statThreadPeriod = 60 * 5;
|
||||
private final AtomicLong chunkCount = new AtomicLong();
|
||||
private final AtomicLong reusedChunkCount = new AtomicLong();
|
||||
private final boolean offheap;
|
||||
|
||||
MemStoreChunkPool(Configuration conf, int chunkSize, int maxCount,
|
||||
int initialCount, float poolSizePercentage) {
|
||||
MemStoreChunkPool(int chunkSize, int maxCount, int initialCount, float poolSizePercentage,
|
||||
boolean offheap) {
|
||||
this.maxCount = maxCount;
|
||||
this.chunkSize = chunkSize;
|
||||
this.poolSizePercentage = poolSizePercentage;
|
||||
this.reclaimedChunks = new LinkedBlockingQueue<PooledChunk>();
|
||||
this.offheap = offheap;
|
||||
this.reclaimedChunks = new LinkedBlockingQueue<>();
|
||||
for (int i = 0; i < initialCount; i++) {
|
||||
PooledChunk chunk = new PooledChunk(chunkSize);
|
||||
Chunk chunk = this.offheap ? new OffheapChunk(chunkSize) : new OnheapChunk(chunkSize);
|
||||
chunk.init();
|
||||
reclaimedChunks.add(chunk);
|
||||
}
|
||||
|
@ -108,8 +103,8 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
* @see #putbackChunk(Chunk)
|
||||
* @see #putbackChunks(BlockingQueue)
|
||||
*/
|
||||
PooledChunk getChunk() {
|
||||
PooledChunk chunk = reclaimedChunks.poll();
|
||||
Chunk getChunk() {
|
||||
Chunk chunk = reclaimedChunks.poll();
|
||||
if (chunk != null) {
|
||||
chunk.reset();
|
||||
reusedChunkCount.incrementAndGet();
|
||||
|
@ -118,7 +113,7 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
while (true) {
|
||||
long created = this.chunkCount.get();
|
||||
if (created < this.maxCount) {
|
||||
chunk = new PooledChunk(chunkSize);
|
||||
chunk = this.offheap ? new OffheapChunk(this.chunkSize) : new OnheapChunk(this.chunkSize);
|
||||
if (this.chunkCount.compareAndSet(created, created + 1)) {
|
||||
break;
|
||||
}
|
||||
|
@ -135,9 +130,9 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
* skip the remaining chunks
|
||||
* @param chunks
|
||||
*/
|
||||
synchronized void putbackChunks(BlockingQueue<PooledChunk> chunks) {
|
||||
synchronized void putbackChunks(BlockingQueue<Chunk> chunks) {
|
||||
int toAdd = Math.min(chunks.size(), this.maxCount - reclaimedChunks.size());
|
||||
PooledChunk chunk = null;
|
||||
Chunk chunk = null;
|
||||
while ((chunk = chunks.poll()) != null && toAdd > 0) {
|
||||
reclaimedChunks.add(chunk);
|
||||
toAdd--;
|
||||
|
@ -149,7 +144,7 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
* skip it
|
||||
* @param chunk
|
||||
*/
|
||||
synchronized void putbackChunk(PooledChunk chunk) {
|
||||
synchronized void putbackChunk(Chunk chunk) {
|
||||
if (reclaimedChunks.size() < this.maxCount) {
|
||||
reclaimedChunks.add(chunk);
|
||||
}
|
||||
|
@ -191,51 +186,41 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param conf
|
||||
* @return the global MemStoreChunkPool instance
|
||||
*/
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DC_DOUBLECHECK",
|
||||
justification="Intentional")
|
||||
static MemStoreChunkPool getPool(Configuration conf) {
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "LI_LAZY_INIT_STATIC",
|
||||
justification = "Method is called by single thread at the starting of RS")
|
||||
static MemStoreChunkPool initialize(long globalMemStoreSize, float poolSizePercentage,
|
||||
float initialCountPercentage, int chunkSize, boolean offheap) {
|
||||
if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
|
||||
if (chunkPoolDisabled) return null;
|
||||
|
||||
synchronized (MemStoreChunkPool.class) {
|
||||
if (chunkPoolDisabled) return null;
|
||||
if (GLOBAL_INSTANCE != null) return GLOBAL_INSTANCE;
|
||||
// When MSLAB is turned OFF no need to init chunk pool at all.
|
||||
if (!conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) {
|
||||
chunkPoolDisabled = true;
|
||||
return null;
|
||||
}
|
||||
float poolSizePercentage = conf.getFloat(CHUNK_POOL_MAXSIZE_KEY, POOL_MAX_SIZE_DEFAULT);
|
||||
if (poolSizePercentage <= 0) {
|
||||
chunkPoolDisabled = true;
|
||||
return null;
|
||||
}
|
||||
if (poolSizePercentage > 1.0) {
|
||||
throw new IllegalArgumentException(CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
|
||||
}
|
||||
long heapMax = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
|
||||
long globalMemStoreLimit = (long) (heapMax * HeapMemorySizeUtil.getGlobalMemStorePercent(conf,
|
||||
false));
|
||||
int chunkSize = conf.getInt(HeapMemStoreLAB.CHUNK_SIZE_KEY,
|
||||
HeapMemStoreLAB.CHUNK_SIZE_DEFAULT);
|
||||
int maxCount = (int) (globalMemStoreLimit * poolSizePercentage / chunkSize);
|
||||
|
||||
float initialCountPercentage = conf.getFloat(CHUNK_POOL_INITIALSIZE_KEY,
|
||||
POOL_INITIAL_SIZE_DEFAULT);
|
||||
if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
|
||||
throw new IllegalArgumentException(CHUNK_POOL_INITIALSIZE_KEY
|
||||
+ " must be between 0.0 and 1.0");
|
||||
}
|
||||
|
||||
int initialCount = (int) (initialCountPercentage * maxCount);
|
||||
LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
|
||||
+ ", max count " + maxCount + ", initial count " + initialCount);
|
||||
GLOBAL_INSTANCE = new MemStoreChunkPool(conf, chunkSize, maxCount, initialCount,
|
||||
poolSizePercentage);
|
||||
return GLOBAL_INSTANCE;
|
||||
if (poolSizePercentage <= 0) {
|
||||
chunkPoolDisabled = true;
|
||||
return null;
|
||||
}
|
||||
if (poolSizePercentage > 1.0) {
|
||||
throw new IllegalArgumentException(
|
||||
MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY + " must be between 0.0 and 1.0");
|
||||
}
|
||||
int maxCount = (int) (globalMemStoreSize * poolSizePercentage / chunkSize);
|
||||
if (initialCountPercentage > 1.0 || initialCountPercentage < 0) {
|
||||
throw new IllegalArgumentException(
|
||||
MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY + " must be between 0.0 and 1.0");
|
||||
}
|
||||
int initialCount = (int) (initialCountPercentage * maxCount);
|
||||
LOG.info("Allocating MemStoreChunkPool with chunk size " + StringUtils.byteDesc(chunkSize)
|
||||
+ ", max count " + maxCount + ", initial count " + initialCount);
|
||||
GLOBAL_INSTANCE = new MemStoreChunkPool(chunkSize, maxCount, initialCount, poolSizePercentage,
|
||||
offheap);
|
||||
return GLOBAL_INSTANCE;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The singleton instance of this pool.
|
||||
*/
|
||||
static MemStoreChunkPool getPool() {
|
||||
return GLOBAL_INSTANCE;
|
||||
}
|
||||
|
||||
int getMaxCount() {
|
||||
|
@ -247,12 +232,6 @@ public class MemStoreChunkPool implements HeapMemoryTuneObserver {
|
|||
chunkPoolDisabled = false;
|
||||
}
|
||||
|
||||
public static class PooledChunk extends Chunk {
|
||||
PooledChunk(int size) {
|
||||
super(size);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onHeapMemoryTune(long newMemstoreSize, long newBlockCacheSize) {
|
||||
int newMaxCount = (int) (newMemstoreSize * poolSizePercentage / chunkSize);
|
||||
|
|
|
@ -24,7 +24,7 @@ import com.google.common.base.Preconditions;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.MemoryType;
|
||||
import java.util.ArrayList;
|
||||
import java.util.ConcurrentModificationException;
|
||||
import java.util.HashMap;
|
||||
|
@ -49,11 +49,12 @@ import org.apache.hadoop.hbase.DroppedSnapshotException;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.HasThread;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
|
@ -109,12 +110,21 @@ class MemStoreFlusher implements FlushRequester {
|
|||
this.conf = conf;
|
||||
this.server = server;
|
||||
this.threadWakeFrequency =
|
||||
conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||
long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
|
||||
float globalMemStorePercent = HeapMemorySizeUtil.getGlobalMemStorePercent(conf, true);
|
||||
this.globalMemStoreLimit = (long) (max * globalMemStorePercent);
|
||||
this.globalMemStoreLimitLowMarkPercent =
|
||||
HeapMemorySizeUtil.getGlobalMemStoreLowerMark(conf, globalMemStorePercent);
|
||||
conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
|
||||
Pair<Long, MemoryType> pair = MemorySizeUtil.getGlobalMemstoreSize(conf);
|
||||
this.globalMemStoreLimit = pair.getFirst();
|
||||
boolean onheap = pair.getSecond() == MemoryType.HEAP;
|
||||
// When off heap memstore in use we configure the global off heap space for memstore as bytes
|
||||
// not as % of max memory size. In such case, the lower water mark should be specified using the
|
||||
// key "hbase.regionserver.global.memstore.size.lower.limit" which says % of the global upper
|
||||
// bound and defaults to 95%. In on heap case also specifying this way is ideal. But in the past
|
||||
// we used to take lower bound also as the % of xmx (38% as default). For backward compatibility
|
||||
// for this deprecated config,we will fall back to read that config when new one is missing.
|
||||
// Only for on heap case, do this fallback mechanism. For off heap it makes no sense.
|
||||
// TODO When to get rid of the deprecated config? ie
|
||||
// "hbase.regionserver.global.memstore.lowerLimit". Can get rid of this boolean passing then.
|
||||
this.globalMemStoreLimitLowMarkPercent = MemorySizeUtil.getGlobalMemStoreHeapLowerMark(conf,
|
||||
onheap);
|
||||
this.globalMemStoreLimitLowMark =
|
||||
(long) (this.globalMemStoreLimit * this.globalMemStoreLimitLowMarkPercent);
|
||||
|
||||
|
@ -126,7 +136,7 @@ class MemStoreFlusher implements FlushRequester {
|
|||
+ TraditionalBinaryPrefix.long2String(this.globalMemStoreLimit, "", 1)
|
||||
+ ", globalMemStoreLimitLowMark="
|
||||
+ TraditionalBinaryPrefix.long2String(this.globalMemStoreLimitLowMark, "", 1)
|
||||
+ ", maxHeap=" + TraditionalBinaryPrefix.long2String(max, "", 1));
|
||||
+ ", Offheap=" + !onheap);
|
||||
}
|
||||
|
||||
public LongAdder getUpdatesBlockedMsHighWater() {
|
||||
|
|
|
@ -17,8 +17,10 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
/**
|
||||
* A memstore-local allocation buffer.
|
||||
|
@ -46,6 +48,19 @@ public interface MemStoreLAB {
|
|||
|
||||
String USEMSLAB_KEY = "hbase.hregion.memstore.mslab.enabled";
|
||||
boolean USEMSLAB_DEFAULT = true;
|
||||
String MSLAB_CLASS_NAME = "hbase.regionserver.mslab.class";
|
||||
|
||||
String CHUNK_SIZE_KEY = "hbase.hregion.memstore.mslab.chunksize";
|
||||
int CHUNK_SIZE_DEFAULT = 2048 * 1024;
|
||||
String MAX_ALLOC_KEY = "hbase.hregion.memstore.mslab.max.allocation";
|
||||
int MAX_ALLOC_DEFAULT = 256 * 1024; // allocs bigger than this don't go through
|
||||
// allocator
|
||||
|
||||
// MSLAB pool related configs
|
||||
String CHUNK_POOL_MAXSIZE_KEY = "hbase.hregion.memstore.chunkpool.maxsize";
|
||||
String CHUNK_POOL_INITIALSIZE_KEY = "hbase.hregion.memstore.chunkpool.initialsize";
|
||||
float POOL_MAX_SIZE_DEFAULT = 1.0f;
|
||||
float POOL_INITIAL_SIZE_DEFAULT = 0.0f;
|
||||
|
||||
/**
|
||||
* Allocates slice in this LAB and copy the passed Cell into this area. Returns new Cell instance
|
||||
|
@ -68,4 +83,17 @@ public interface MemStoreLAB {
|
|||
*/
|
||||
void decScannerCount();
|
||||
|
||||
public static MemStoreLAB newInstance(Configuration conf) {
|
||||
MemStoreLAB memStoreLAB = null;
|
||||
if (isEnabled(conf)) {
|
||||
String className = conf.get(MSLAB_CLASS_NAME, MemStoreLABImpl.class.getName());
|
||||
memStoreLAB = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
new Class[] { Configuration.class }, new Object[] { conf });
|
||||
}
|
||||
return memStoreLAB;
|
||||
}
|
||||
|
||||
public static boolean isEnabled(Configuration conf) {
|
||||
return conf.getBoolean(USEMSLAB_KEY, USEMSLAB_DEFAULT);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,9 +28,9 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.MemStoreChunkPool.PooledChunk;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -43,7 +43,7 @@ import com.google.common.base.Preconditions;
|
|||
* slices into the array.
|
||||
* <p>
|
||||
* The purpose of this class is to combat heap fragmentation in the
|
||||
* regionserver. By ensuring that all KeyValues in a given memstore refer
|
||||
* regionserver. By ensuring that all Cells in a given memstore refer
|
||||
* only to large chunks of contiguous memory, we ensure that large blocks
|
||||
* get freed up when the memstore is flushed.
|
||||
* <p>
|
||||
|
@ -54,25 +54,23 @@ import com.google.common.base.Preconditions;
|
|||
* TODO: we should probably benchmark whether word-aligning the allocations
|
||||
* would provide a performance improvement - probably would speed up the
|
||||
* Bytes.toLong/Bytes.toInt calls in KeyValue, but some of those are cached
|
||||
* anyway
|
||||
* anyway.
|
||||
* The chunks created by this MemStoreLAB can get pooled at {@link MemStoreChunkPool}.
|
||||
* When the Chunk comes pool, it can be either an on heap or an off heap backed chunk. The chunks,
|
||||
* which this MemStoreLAB creates on its own (when no chunk available from pool), those will be
|
||||
* always on heap backed.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HeapMemStoreLAB implements MemStoreLAB {
|
||||
public class MemStoreLABImpl implements MemStoreLAB {
|
||||
|
||||
static final String CHUNK_SIZE_KEY = "hbase.hregion.memstore.mslab.chunksize";
|
||||
static final int CHUNK_SIZE_DEFAULT = 2048 * 1024;
|
||||
static final String MAX_ALLOC_KEY = "hbase.hregion.memstore.mslab.max.allocation";
|
||||
static final int MAX_ALLOC_DEFAULT = 256 * 1024; // allocs bigger than this don't go through
|
||||
// allocator
|
||||
|
||||
static final Log LOG = LogFactory.getLog(HeapMemStoreLAB.class);
|
||||
static final Log LOG = LogFactory.getLog(MemStoreLABImpl.class);
|
||||
|
||||
private AtomicReference<Chunk> curChunk = new AtomicReference<Chunk>();
|
||||
// A queue of chunks from pool contained by this memstore LAB
|
||||
// TODO: in the future, it would be better to have List implementation instead of Queue,
|
||||
// as FIFO order is not so important here
|
||||
@VisibleForTesting
|
||||
BlockingQueue<PooledChunk> pooledChunkQueue = null;
|
||||
BlockingQueue<Chunk> pooledChunkQueue = null;
|
||||
private final int chunkSize;
|
||||
private final int maxAlloc;
|
||||
private final MemStoreChunkPool chunkPool;
|
||||
|
@ -87,19 +85,19 @@ public class HeapMemStoreLAB implements MemStoreLAB {
|
|||
private final AtomicInteger openScannerCount = new AtomicInteger();
|
||||
|
||||
// Used in testing
|
||||
public HeapMemStoreLAB() {
|
||||
public MemStoreLABImpl() {
|
||||
this(new Configuration());
|
||||
}
|
||||
|
||||
public HeapMemStoreLAB(Configuration conf) {
|
||||
public MemStoreLABImpl(Configuration conf) {
|
||||
chunkSize = conf.getInt(CHUNK_SIZE_KEY, CHUNK_SIZE_DEFAULT);
|
||||
maxAlloc = conf.getInt(MAX_ALLOC_KEY, MAX_ALLOC_DEFAULT);
|
||||
this.chunkPool = MemStoreChunkPool.getPool(conf);
|
||||
this.chunkPool = MemStoreChunkPool.getPool();
|
||||
// currently chunkQueue is only used for chunkPool
|
||||
if (this.chunkPool != null) {
|
||||
// set queue length to chunk pool max count to avoid keeping reference of
|
||||
// too many non-reclaimable chunks
|
||||
pooledChunkQueue = new LinkedBlockingQueue<PooledChunk>(chunkPool.getMaxCount());
|
||||
pooledChunkQueue = new LinkedBlockingQueue<>(chunkPool.getMaxCount());
|
||||
}
|
||||
|
||||
// if we don't exclude allocations >CHUNK_SIZE, we'd infiniteloop on one!
|
||||
|
@ -132,7 +130,7 @@ public class HeapMemStoreLAB implements MemStoreLAB {
|
|||
// try to retire this chunk
|
||||
tryRetireChunk(c);
|
||||
}
|
||||
return KeyValueUtil.copyCellTo(cell, c.getData(), allocOffset, size);
|
||||
return CellUtil.copyCellTo(cell, c.getData(), allocOffset, size);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -210,14 +208,14 @@ public class HeapMemStoreLAB implements MemStoreLAB {
|
|||
// This is chunk from pool
|
||||
pooledChunk = true;
|
||||
} else {
|
||||
c = new Chunk(chunkSize);
|
||||
c = new OnheapChunk(chunkSize);// When chunk is not from pool, always make it as on heap.
|
||||
}
|
||||
if (curChunk.compareAndSet(null, c)) {
|
||||
// we won race - now we need to actually do the expensive
|
||||
// allocation step
|
||||
c.init();
|
||||
if (pooledChunk) {
|
||||
if (!this.closed && !this.pooledChunkQueue.offer((PooledChunk) c)) {
|
||||
if (!this.closed && !this.pooledChunkQueue.offer(c)) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Chunk queue is full, won't reuse this new chunk. Current queue size: "
|
||||
+ pooledChunkQueue.size());
|
||||
|
@ -226,7 +224,7 @@ public class HeapMemStoreLAB implements MemStoreLAB {
|
|||
}
|
||||
return c;
|
||||
} else if (pooledChunk) {
|
||||
chunkPool.putbackChunk((PooledChunk) c);
|
||||
chunkPool.putbackChunk(c);
|
||||
}
|
||||
// someone else won race - that's fine, we'll try to grab theirs
|
||||
// in the next iteration of the loop.
|
||||
|
@ -239,7 +237,7 @@ public class HeapMemStoreLAB implements MemStoreLAB {
|
|||
}
|
||||
|
||||
|
||||
BlockingQueue<PooledChunk> getPooledChunks() {
|
||||
BlockingQueue<Chunk> getPooledChunks() {
|
||||
return this.pooledChunkQueue;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
* 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 java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* An off heap chunk implementation.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class OffheapChunk extends Chunk {
|
||||
|
||||
OffheapChunk(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init() {
|
||||
assert nextFreeOffset.get() == UNINITIALIZED;
|
||||
try {
|
||||
if (data == null) {
|
||||
data = ByteBuffer.allocateDirect(this.size);
|
||||
}
|
||||
} catch (OutOfMemoryError e) {
|
||||
boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
|
||||
assert failInit; // should be true.
|
||||
throw e;
|
||||
}
|
||||
// Mark that it's ready for use
|
||||
boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
|
||||
// We should always succeed the above CAS since only one thread
|
||||
// calls init()!
|
||||
Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,53 @@
|
|||
/**
|
||||
* 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 java.nio.ByteBuffer;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* An on heap chunk implementation.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class OnheapChunk extends Chunk {
|
||||
|
||||
OnheapChunk(int size) {
|
||||
super(size);
|
||||
}
|
||||
|
||||
public void init() {
|
||||
assert nextFreeOffset.get() == UNINITIALIZED;
|
||||
try {
|
||||
if (data == null) {
|
||||
data = ByteBuffer.allocate(this.size);
|
||||
}
|
||||
} catch (OutOfMemoryError e) {
|
||||
boolean failInit = nextFreeOffset.compareAndSet(UNINITIALIZED, OOM);
|
||||
assert failInit; // should be true.
|
||||
throw e;
|
||||
}
|
||||
// Mark that it's ready for use
|
||||
boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
|
||||
// We should always succeed the above CAS since only one thread
|
||||
// calls init()!
|
||||
Preconditions.checkState(initted, "Multiple threads tried to init same chunk");
|
||||
}
|
||||
}
|
|
@ -45,7 +45,7 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferedCell;
|
||||
import org.apache.hadoop.hbase.ByteBufferCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScannable;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
|
@ -1174,8 +1174,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
// Since byte buffers can point all kinds of crazy places it's harder to keep track
|
||||
// of which blocks are kept alive by what byte buffer.
|
||||
// So we make a guess.
|
||||
if (c instanceof ByteBufferedCell) {
|
||||
ByteBufferedCell bbCell = (ByteBufferedCell) c;
|
||||
if (c instanceof ByteBufferCell) {
|
||||
ByteBufferCell bbCell = (ByteBufferCell) c;
|
||||
ByteBuffer bb = bbCell.getValueByteBuffer();
|
||||
if (bb != lastBlock) {
|
||||
context.incrementResponseBlockSize(bb.capacity());
|
||||
|
|
|
@ -22,7 +22,6 @@ import com.google.common.base.Preconditions;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -35,8 +34,6 @@ import java.util.List;
|
|||
@InterfaceAudience.Private
|
||||
public final class SegmentFactory {
|
||||
|
||||
static final String MSLAB_CLASS_NAME = "hbase.regionserver.mslab.class";
|
||||
|
||||
private SegmentFactory() {}
|
||||
private static SegmentFactory instance = new SegmentFactory();
|
||||
|
||||
|
@ -47,7 +44,7 @@ public final class SegmentFactory {
|
|||
// create skip-list-based (non-flat) immutable segment from compacting old immutable segments
|
||||
public ImmutableSegment createImmutableSegment(final Configuration conf,
|
||||
final CellComparator comparator, MemStoreSegmentsIterator iterator) {
|
||||
return new ImmutableSegment(comparator, iterator, getMemStoreLAB(conf));
|
||||
return new ImmutableSegment(comparator, iterator, MemStoreLAB.newInstance(conf));
|
||||
}
|
||||
|
||||
// create new flat immutable segment from compacting old immutable segments
|
||||
|
@ -57,7 +54,7 @@ public final class SegmentFactory {
|
|||
throws IOException {
|
||||
Preconditions.checkArgument(segmentType == ImmutableSegment.Type.ARRAY_MAP_BASED,
|
||||
"wrong immutable segment type");
|
||||
MemStoreLAB memStoreLAB = getMemStoreLAB(conf);
|
||||
MemStoreLAB memStoreLAB = MemStoreLAB.newInstance(conf);
|
||||
return
|
||||
// the last parameter "false" means not to merge, but to compact the pipeline
|
||||
// in order to create the new segment
|
||||
|
@ -77,7 +74,7 @@ public final class SegmentFactory {
|
|||
|
||||
// create mutable segment
|
||||
public MutableSegment createMutableSegment(final Configuration conf, CellComparator comparator) {
|
||||
MemStoreLAB memStoreLAB = getMemStoreLAB(conf);
|
||||
MemStoreLAB memStoreLAB = MemStoreLAB.newInstance(conf);
|
||||
return generateMutableSegment(conf, comparator, memStoreLAB);
|
||||
}
|
||||
|
||||
|
@ -103,16 +100,6 @@ public final class SegmentFactory {
|
|||
return new MutableSegment(set, comparator, memStoreLAB);
|
||||
}
|
||||
|
||||
private MemStoreLAB getMemStoreLAB(Configuration conf) {
|
||||
MemStoreLAB memStoreLAB = null;
|
||||
if (conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) {
|
||||
String className = conf.get(MSLAB_CLASS_NAME, HeapMemStoreLAB.class.getName());
|
||||
memStoreLAB = ReflectionUtils.instantiateWithCustomCtor(className,
|
||||
new Class[] { Configuration.class }, new Object[] { conf });
|
||||
}
|
||||
return memStoreLAB;
|
||||
}
|
||||
|
||||
private MemStoreLAB getMergedMemStoreLAB(Configuration conf, List<ImmutableSegment> segments) {
|
||||
List<MemStoreLAB> mslabs = new ArrayList<MemStoreLAB>();
|
||||
for (ImmutableSegment segment : segments) {
|
||||
|
|
|
@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.HConstants;
|
|||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.CollectionUtils;
|
||||
import org.apache.hadoop.hbase.util.DrainBarrier;
|
||||
|
@ -386,8 +386,7 @@ public abstract class AbstractFSWAL<W> implements WAL {
|
|||
this.logrollsize = (long) (blocksize
|
||||
* conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
|
||||
|
||||
float memstoreRatio = conf.getFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, conf.getFloat(
|
||||
HeapMemorySizeUtil.MEMSTORE_SIZE_OLD_KEY, HeapMemorySizeUtil.DEFAULT_MEMSTORE_SIZE));
|
||||
float memstoreRatio = MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false);
|
||||
boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null;
|
||||
if (maxLogsDefined) {
|
||||
LOG.warn("'hbase.regionserver.maxlogs' was deprecated.");
|
||||
|
|
|
@ -27,7 +27,7 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferedCellImpl;
|
||||
import org.apache.hadoop.hbase.TestCellUtil.ByteBufferCellImpl;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.testclassification.FilterTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -108,7 +108,7 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("less than", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("less than", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
|
||||
|
@ -117,7 +117,7 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("Equals 100", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("Equals 100", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
filter.reset();
|
||||
|
||||
|
@ -126,7 +126,7 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("include 120", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
filter.reset();
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("include 120", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
}
|
||||
|
||||
|
@ -135,29 +135,29 @@ public class TestSingleColumnValueFilter {
|
|||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_3);
|
||||
assertTrue("basicFilter2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_4);
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter3", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("basicFilterNotNull", filter.filterRow());
|
||||
filter.reset();
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(kv) == Filter.ReturnCode.NEXT_ROW);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter4", filter.filterKeyValue(c) == Filter.ReturnCode.NEXT_ROW);
|
||||
assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertTrue("basicFilterNotNull", filter.filterRow());
|
||||
|
@ -166,12 +166,12 @@ public class TestSingleColumnValueFilter {
|
|||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_1);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("basicFilter5", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("basicFilterNotNull", filter.filterRow());
|
||||
}
|
||||
|
@ -181,14 +181,14 @@ public class TestSingleColumnValueFilter {
|
|||
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, FULLSTRING_1);
|
||||
assertTrue("null1", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("null1", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("null1FilterRow", filter.filterRow());
|
||||
filter.reset();
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, Bytes.toBytes("qual2"), FULLSTRING_2);
|
||||
assertTrue("null2", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("null2", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertTrue("null2FilterRow", filter.filterRow());
|
||||
}
|
||||
|
@ -200,13 +200,13 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("substrTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("substrTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertTrue("substrFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("substrFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("substrFilterNotNull", filter.filterRow());
|
||||
|
@ -219,13 +219,13 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("regexTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
|
||||
FULLSTRING_2);
|
||||
assertTrue("regexFalse", filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
buffer = kv.getBuffer();
|
||||
c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexFalse", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("regexFilterNotNull", filter.filterRow());
|
||||
|
@ -238,7 +238,7 @@ public class TestSingleColumnValueFilter {
|
|||
assertTrue("regexTrue",
|
||||
filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
|
||||
byte[] buffer = kv.getBuffer();
|
||||
Cell c = new ByteBufferedCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
Cell c = new ByteBufferCellImpl(ByteBuffer.wrap(buffer), 0, buffer.length);
|
||||
assertTrue("regexTrue", filter.filterKeyValue(c) == Filter.ReturnCode.INCLUDE);
|
||||
assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
|
||||
assertFalse("regexFilterNotNull", filter.filterRow());
|
||||
|
|
|
@ -44,7 +44,6 @@ public class TestCellFlatSet extends TestCase {
|
|||
private Cell descCells[];
|
||||
private CellArrayMap descCbOnHeap;
|
||||
private final static Configuration CONF = new Configuration();
|
||||
private HeapMemStoreLAB mslab;
|
||||
private KeyValue lowerOuterCell;
|
||||
private KeyValue upperOuterCell;
|
||||
|
||||
|
@ -73,9 +72,8 @@ public class TestCellFlatSet extends TestCase {
|
|||
descCells = new Cell[] {kv4,kv3,kv2,kv1};
|
||||
descCbOnHeap = new CellArrayMap(CellComparator.COMPARATOR,descCells,0,NUM_OF_CELLS,true);
|
||||
CONF.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
|
||||
CONF.setFloat(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
CONF.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
MemStoreChunkPool.chunkPoolDisabled = false;
|
||||
mslab = new HeapMemStoreLAB(CONF);
|
||||
}
|
||||
|
||||
/* Create and test CellSet based on CellArrayMap */
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -35,6 +36,7 @@ import org.apache.hadoop.hbase.KeepDeletedCells;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueTestUtil;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -87,7 +89,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
|
|||
super.internalSetUp();
|
||||
Configuration conf = new Configuration();
|
||||
conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
|
||||
conf.setFloat(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
conf.setInt(HRegion.MEMSTORE_PERIODIC_FLUSH_INTERVAL, 1000);
|
||||
HBaseTestingUtility hbaseUtility = HBaseTestingUtility.createLocalHTU(conf);
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(FAMILY);
|
||||
|
@ -95,7 +97,10 @@ public class TestCompactingMemStore extends TestDefaultMemStore {
|
|||
this.regionServicesForStores = region.getRegionServicesForStores();
|
||||
this.store = new HStore(region, hcd, conf);
|
||||
|
||||
chunkPool = MemStoreChunkPool.getPool(conf);
|
||||
long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
|
||||
.getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
|
||||
chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
|
||||
MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
|
||||
assertTrue(chunkPool != null);
|
||||
}
|
||||
|
||||
|
|
|
@ -132,9 +132,9 @@ public class TestDefaultMemStore {
|
|||
// make sure memstore size increased even when writing the same cell, if using MSLAB
|
||||
assertEquals(Segment.getCellLength(kv), sizeChangeForSecondCell.getDataSize());
|
||||
// make sure chunk size increased even when writing the same cell, if using MSLAB
|
||||
if (msLab instanceof HeapMemStoreLAB) {
|
||||
if (msLab instanceof MemStoreLABImpl) {
|
||||
assertEquals(2 * Segment.getCellLength(kv),
|
||||
((HeapMemStoreLAB) msLab).getCurrentChunk().getNextFreeOffset());
|
||||
((MemStoreLABImpl) msLab).getCurrentChunk().getNextFreeOffset());
|
||||
}
|
||||
} else {
|
||||
// make sure no memstore size change w/o MSLAB
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheStats;
|
|||
import org.apache.hadoop.hbase.io.hfile.Cacheable;
|
||||
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
|
||||
import org.apache.hadoop.hbase.io.hfile.ResizableBlockCache;
|
||||
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerContext;
|
||||
import org.apache.hadoop.hbase.regionserver.HeapMemoryManager.TunerResult;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
|
@ -62,7 +62,7 @@ public class TestHeapMemoryManager {
|
|||
@Test
|
||||
public void testAutoTunerShouldBeOffWhenMaxMinRangesForMemstoreIsNotGiven() throws Exception {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.setFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, 0.02f);
|
||||
conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.02f);
|
||||
conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.75f);
|
||||
conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MIN_RANGE_KEY, 0.03f);
|
||||
HeapMemoryManager manager = new HeapMemoryManager(new BlockCacheStub(0),
|
||||
|
@ -228,7 +228,7 @@ public class TestHeapMemoryManager {
|
|||
blockCache.setTestBlockSize((long) (maxHeapSize * 0.4 * 0.8));
|
||||
regionServerAccounting.setTestMemstoreSize(0);
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.setFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_LOWER_LIMIT_KEY, 0.7f);
|
||||
conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_LOWER_LIMIT_KEY, 0.7f);
|
||||
conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MAX_RANGE_KEY, 0.75f);
|
||||
conf.setFloat(HeapMemoryManager.MEMSTORE_SIZE_MIN_RANGE_KEY, 0.10f);
|
||||
conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f);
|
||||
|
@ -462,7 +462,7 @@ public class TestHeapMemoryManager {
|
|||
conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MAX_RANGE_KEY, 0.7f);
|
||||
conf.setFloat(HeapMemoryManager.BLOCK_CACHE_SIZE_MIN_RANGE_KEY, 0.1f);
|
||||
conf.setInt(DefaultHeapMemoryTuner.NUM_PERIODS_TO_IGNORE, 0);
|
||||
conf.setFloat(HeapMemorySizeUtil.MEMSTORE_SIZE_KEY, 0.4F);
|
||||
conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.4F);
|
||||
conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.3F);
|
||||
conf.setFloat(HConstants.BUCKET_CACHE_SIZE_KEY, 0.1F);
|
||||
conf.set(HConstants.BUCKET_CACHE_IOENGINE_KEY, "heap");
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -32,6 +33,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
|
@ -50,10 +52,13 @@ public class TestMemStoreChunkPool {
|
|||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
conf.setBoolean(MemStoreLAB.USEMSLAB_KEY, true);
|
||||
conf.setFloat(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
conf.setFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.2f);
|
||||
chunkPoolDisabledBeforeTest = MemStoreChunkPool.chunkPoolDisabled;
|
||||
MemStoreChunkPool.chunkPoolDisabled = false;
|
||||
chunkPool = MemStoreChunkPool.getPool(conf);
|
||||
long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
|
||||
.getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
|
||||
chunkPool = MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f,
|
||||
MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT, MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
|
||||
assertTrue(chunkPool != null);
|
||||
}
|
||||
|
||||
|
@ -70,7 +75,7 @@ public class TestMemStoreChunkPool {
|
|||
@Test
|
||||
public void testReusingChunks() {
|
||||
Random rand = new Random();
|
||||
MemStoreLAB mslab = new HeapMemStoreLAB(conf);
|
||||
MemStoreLAB mslab = new MemStoreLABImpl(conf);
|
||||
int expectedOff = 0;
|
||||
byte[] lastBuffer = null;
|
||||
final byte[] rk = Bytes.toBytes("r1");
|
||||
|
@ -96,7 +101,7 @@ public class TestMemStoreChunkPool {
|
|||
int chunkCount = chunkPool.getPoolSize();
|
||||
assertTrue(chunkCount > 0);
|
||||
// reconstruct mslab
|
||||
mslab = new HeapMemStoreLAB(conf);
|
||||
mslab = new MemStoreLABImpl(conf);
|
||||
// chunk should be got from the pool, so we can reuse it.
|
||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[10]);
|
||||
mslab.copyCellInto(kv);
|
||||
|
@ -209,7 +214,7 @@ public class TestMemStoreChunkPool {
|
|||
final int initialCount = 5;
|
||||
final int chunkSize = 30;
|
||||
final int valSize = 7;
|
||||
MemStoreChunkPool pool = new MemStoreChunkPool(conf, chunkSize, maxCount, initialCount, 1);
|
||||
MemStoreChunkPool pool = new MemStoreChunkPool(chunkSize, maxCount, initialCount, 1, false);
|
||||
assertEquals(initialCount, pool.getPoolSize());
|
||||
assertEquals(maxCount, pool.getMaxCount());
|
||||
MemStoreChunkPool.GLOBAL_INSTANCE = pool;// Replace the global ref with the new one we created.
|
||||
|
@ -221,7 +226,7 @@ public class TestMemStoreChunkPool {
|
|||
Runnable r = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
MemStoreLAB memStoreLAB = new HeapMemStoreLAB(conf);
|
||||
MemStoreLAB memStoreLAB = new MemStoreLABImpl(conf);
|
||||
for (int i = 0; i < maxCount; i++) {
|
||||
memStoreLAB.copyCellInto(kv);// Try allocate size = chunkSize. Means every
|
||||
// allocate call will result in a new chunk
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -33,9 +34,11 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil;
|
||||
import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
|
||||
import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -48,17 +51,27 @@ import org.junit.experimental.categories.Category;
|
|||
@Category({RegionServerTests.class, SmallTests.class})
|
||||
public class TestMemStoreLAB {
|
||||
|
||||
private final static Configuration conf = new Configuration();
|
||||
|
||||
private static final byte[] rk = Bytes.toBytes("r1");
|
||||
private static final byte[] cf = Bytes.toBytes("f");
|
||||
private static final byte[] q = Bytes.toBytes("q");
|
||||
|
||||
@BeforeClass
|
||||
public static void setUpBeforeClass() throws Exception {
|
||||
long globalMemStoreLimit = (long) (ManagementFactory.getMemoryMXBean().getHeapMemoryUsage()
|
||||
.getMax() * MemorySizeUtil.getGlobalMemStoreHeapPercent(conf, false));
|
||||
MemStoreChunkPool.initialize(globalMemStoreLimit, 0.2f, MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT,
|
||||
MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test a bunch of random allocations
|
||||
*/
|
||||
@Test
|
||||
public void testLABRandomAllocation() {
|
||||
Random rand = new Random();
|
||||
MemStoreLAB mslab = new HeapMemStoreLAB();
|
||||
MemStoreLAB mslab = new MemStoreLABImpl();
|
||||
int expectedOff = 0;
|
||||
byte[] lastBuffer = null;
|
||||
// 100K iterations by 0-1K alloc -> 50MB expected
|
||||
|
@ -82,7 +95,7 @@ public class TestMemStoreLAB {
|
|||
|
||||
@Test
|
||||
public void testLABLargeAllocation() {
|
||||
MemStoreLAB mslab = new HeapMemStoreLAB();
|
||||
MemStoreLAB mslab = new MemStoreLABImpl();
|
||||
KeyValue kv = new KeyValue(rk, cf, q, new byte[2 * 1024 * 1024]);
|
||||
Cell newCell = mslab.copyCellInto(kv);
|
||||
assertNull("2MB allocation shouldn't be satisfied by LAB.", newCell);
|
||||
|
@ -100,7 +113,7 @@ public class TestMemStoreLAB {
|
|||
|
||||
final AtomicInteger totalAllocated = new AtomicInteger();
|
||||
|
||||
final MemStoreLAB mslab = new HeapMemStoreLAB();
|
||||
final MemStoreLAB mslab = new MemStoreLABImpl();
|
||||
List<List<AllocRecord>> allocations = Lists.newArrayList();
|
||||
|
||||
for (int i = 0; i < 10; i++) {
|
||||
|
@ -170,21 +183,21 @@ public class TestMemStoreLAB {
|
|||
*/
|
||||
@Test
|
||||
public void testLABChunkQueue() throws Exception {
|
||||
HeapMemStoreLAB mslab = new HeapMemStoreLAB();
|
||||
MemStoreLABImpl mslab = new MemStoreLABImpl();
|
||||
// by default setting, there should be no chunks initialized in the pool
|
||||
assertTrue(mslab.getPooledChunks().isEmpty());
|
||||
// reset mslab with chunk pool
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.setDouble(MemStoreChunkPool.CHUNK_POOL_MAXSIZE_KEY, 0.1);
|
||||
conf.setDouble(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, 0.1);
|
||||
// set chunk size to default max alloc size, so we could easily trigger chunk retirement
|
||||
conf.setLong(HeapMemStoreLAB.CHUNK_SIZE_KEY, HeapMemStoreLAB.MAX_ALLOC_DEFAULT);
|
||||
conf.setLong(MemStoreLABImpl.CHUNK_SIZE_KEY, MemStoreLABImpl.MAX_ALLOC_DEFAULT);
|
||||
// reconstruct mslab
|
||||
MemStoreChunkPool.clearDisableFlag();
|
||||
mslab = new HeapMemStoreLAB(conf);
|
||||
mslab = new MemStoreLABImpl(conf);
|
||||
// launch multiple threads to trigger frequent chunk retirement
|
||||
List<Thread> threads = new ArrayList<Thread>();
|
||||
final KeyValue kv = new KeyValue(Bytes.toBytes("r"), Bytes.toBytes("f"), Bytes.toBytes("q"),
|
||||
new byte[HeapMemStoreLAB.MAX_ALLOC_DEFAULT - 24]);
|
||||
new byte[MemStoreLABImpl.MAX_ALLOC_DEFAULT - 24]);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
threads.add(getChunkQueueTestThread(mslab, "testLABChunkQueue-" + i, kv));
|
||||
}
|
||||
|
@ -214,7 +227,7 @@ public class TestMemStoreLAB {
|
|||
+ " after mslab closed but actually: " + queueLength, queueLength == 0);
|
||||
}
|
||||
|
||||
private Thread getChunkQueueTestThread(final HeapMemStoreLAB mslab, String threadName,
|
||||
private Thread getChunkQueueTestThread(final MemStoreLABImpl mslab, String threadName,
|
||||
Cell cellToCopyInto) {
|
||||
Thread thread = new Thread() {
|
||||
boolean stopped = false;
|
||||
|
|
Loading…
Reference in New Issue