HBASE-14144 - Bloomfilter path to work with Byte buffered cells (Ram)

This commit is contained in:
ramkrishna 2015-08-14 12:00:27 +05:30
parent 4dd30ab019
commit f2a9dab30e
4 changed files with 312 additions and 63 deletions

View File

@ -1225,9 +1225,30 @@ public final class CellUtil {
* @return First possible Cell on passed Cell's row.
*/
public static Cell createFirstOnRow(final Cell cell) {
return new FirstOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
if (cell instanceof ByteBufferedCell) {
return new FirstOnRowByteBufferedCell(((ByteBufferedCell) cell).getRowByteBuffer(),
((ByteBufferedCell) cell).getRowPositionInByteBuffer(), cell.getRowLength());
}
return new FirstOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
}
/**
* Create a Cell that is smaller than all other possible Cells for the given Cell's row.
*
* @param cell
* @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).getRowPositionInByteBuffer(), cell.getRowLength(),
((ByteBufferedCell) cell).getQualifierByteBuffer(),
((ByteBufferedCell) cell).getQualifierPositionInByteBuffer(), cell.getQualifierLength());
}
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength(), HConstants.EMPTY_BYTE_ARRAY, 0, (byte)0, cell.getQualifierArray(),
cell.getQualifierOffset(), cell.getQualifierLength());
}
/**
* Create a Cell that is smaller than all other possible Cells for the given Cell row's next row.
* Makes the next row's rowkey by appending single byte 0x00 to the end of current row key.
@ -1236,7 +1257,7 @@ public final class CellUtil {
byte[] nextRow = new byte[cell.getRowLength() + 1];
copyRowTo(cell, nextRow, 0);
nextRow[nextRow.length - 1] = 0;// maybe not necessary
return new FirstOnRowFakeCell(nextRow, 0, (short) nextRow.length);
return new FirstOnRowCell(nextRow, 0, (short) nextRow.length);
}
/**
@ -1250,7 +1271,7 @@ 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) {
return new FirstOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(),
return new FirstOnRowColCell(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
qArray, qoffest, qlength);
}
@ -1264,7 +1285,7 @@ public final class CellUtil {
* @param ts
*/
public static Cell createFirstOnRowColTS(Cell cell, long ts) {
return new FirstOnRowColumnTSFakeCell(cell.getRowArray(), cell.getRowOffset(),
return new FirstOnRowColTSCell(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength(), cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(), ts);
}
@ -1276,7 +1297,7 @@ public final class CellUtil {
* @return Last possible Cell on passed Cell's row.
*/
public static Cell createLastOnRow(final Cell cell) {
return new LastOnRowFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
return new LastOnRowCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
}
/**
@ -1288,7 +1309,7 @@ public final class CellUtil {
* @return Last possible Cell on passed Cell's rk:cf:q.
*/
public static Cell createLastOnRowCol(final Cell cell) {
return new LastOnRowColumnFakeCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
return new LastOnRowColCell(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
}
@ -1307,7 +1328,11 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static abstract class FakeCell implements Cell {
/**
* 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 EmptyCell implements Cell {
@Override
public byte[] getRowArray() {
@ -1391,12 +1416,150 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class FirstOnRowFakeCell extends FakeCell {
/**
* 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 {
@Override
public byte[] getRowArray() {
return CellUtil.cloneRow(this);
}
@Override
public int getRowOffset() {
return 0;
}
@Override
public short getRowLength() {
return 0;
}
@Override
public byte[] getFamilyArray() {
return CellUtil.cloneFamily(this);
}
@Override
public int getFamilyOffset() {
return 0;
}
@Override
public byte getFamilyLength() {
return 0;
}
@Override
public byte[] getQualifierArray() {
return CellUtil.cloneQualifier(this);
}
@Override
public int getQualifierOffset() {
return 0;
}
@Override
public int getQualifierLength() {
return 0;
}
@Override
public long getSequenceId() {
return 0;
}
@Override
public byte[] getValueArray() {
return CellUtil.cloneValue(this);
}
@Override
public int getValueOffset() {
return 0;
}
@Override
public int getValueLength() {
return 0;
}
@Override
public byte[] getTagsArray() {
return CellUtil.cloneTags(this);
}
@Override
public int getTagsOffset() {
return 0;
}
@Override
public int getTagsLength() {
return 0;
}
@Override
public ByteBuffer getRowByteBuffer() {
return HConstants.EMPTY_BYTE_BUFFER;
}
@Override
public int getRowPositionInByteBuffer() {
return 0;
}
@Override
public ByteBuffer getFamilyByteBuffer() {
return HConstants.EMPTY_BYTE_BUFFER;
}
@Override
public int getFamilyPositionInByteBuffer() {
return 0;
}
@Override
public ByteBuffer getQualifierByteBuffer() {
return HConstants.EMPTY_BYTE_BUFFER;
}
@Override
public int getQualifierPositionInByteBuffer() {
return 0;
}
@Override
public ByteBuffer getTagsByteBuffer() {
return HConstants.EMPTY_BYTE_BUFFER;
}
@Override
public int getTagsPositionInByteBuffer() {
return 0;
}
@Override
public ByteBuffer getValueByteBuffer() {
return HConstants.EMPTY_BYTE_BUFFER;
}
@Override
public int getValuePositionInByteBuffer() {
return 0;
}
}
@InterfaceAudience.Private
private static class FirstOnRowCell extends EmptyCell {
private final byte[] rowArray;
private final int roffset;
private final short rlength;
public FirstOnRowFakeCell(final byte[] row, int roffset, short rlength) {
public FirstOnRowCell(final byte[] row, int roffset, short rlength) {
this.rowArray = row;
this.roffset = roffset;
this.rlength = rlength;
@ -1429,7 +1592,85 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class FirstOnRowColumnFakeCell extends FirstOnRowFakeCell {
private static class FirstOnRowByteBufferedCell extends EmptyByteBufferedCell {
private final ByteBuffer rowBuff;
private final int roffset;
private final short rlength;
public FirstOnRowByteBufferedCell(final ByteBuffer row, int roffset, short rlength) {
this.rowBuff = row;
this.roffset = roffset;
this.rlength = rlength;
}
@Override
public ByteBuffer getRowByteBuffer() {
return this.rowBuff;
}
@Override
public int getRowPositionInByteBuffer() {
return this.roffset;
}
@Override
public short getRowLength() {
return this.rlength;
}
@Override
public long getTimestamp() {
return HConstants.LATEST_TIMESTAMP;
}
@Override
public byte getTypeByte() {
return Type.Maximum.getCode();
}
}
@InterfaceAudience.Private
private static class FirstOnRowColByteBufferedCell extends FirstOnRowByteBufferedCell {
private final ByteBuffer colBuff;
private final int colOffset;
private final int colLength;
public FirstOnRowColByteBufferedCell(final ByteBuffer row, int roffset, short rlength,
final ByteBuffer col, final int colOffset, final int colLength) {
super(row, roffset, rlength);
this.colBuff = col;
this.colOffset = colOffset;
this.colLength = colLength;
}
@Override
public ByteBuffer getQualifierByteBuffer() {
return this.colBuff;
}
@Override
public int getQualifierPositionInByteBuffer() {
return this.colOffset;
}
@Override
public int getQualifierLength() {
return this.colLength;
}
@Override
public long getTimestamp() {
return HConstants.LATEST_TIMESTAMP;
}
@Override
public byte getTypeByte() {
return Type.Maximum.getCode();
}
}
@InterfaceAudience.Private
private static class FirstOnRowColCell extends FirstOnRowCell {
private final byte[] fArray;
private final int foffset;
private final byte flength;
@ -1437,7 +1678,7 @@ public final class CellUtil {
private final int qoffset;
private final int qlength;
public FirstOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
public FirstOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
super(rArray, roffset, rlength);
this.fArray = fArray;
@ -1480,11 +1721,11 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class FirstOnRowColumnTSFakeCell extends FirstOnRowColumnFakeCell {
private static class FirstOnRowColTSCell extends FirstOnRowColCell {
private long ts;
public FirstOnRowColumnTSFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
public FirstOnRowColTSCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
int foffset, byte flength, byte[] qArray, int qoffset, int qlength, long ts) {
super(rArray, roffset, rlength, fArray, foffset, flength, qArray, qoffset, qlength);
this.ts = ts;
@ -1497,12 +1738,12 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class LastOnRowFakeCell extends FakeCell {
private static class LastOnRowCell extends EmptyCell {
private final byte[] rowArray;
private final int roffset;
private final short rlength;
public LastOnRowFakeCell(byte[] row, int roffset, short rlength) {
public LastOnRowCell(byte[] row, int roffset, short rlength) {
this.rowArray = row;
this.roffset = roffset;
this.rlength = rlength;
@ -1535,7 +1776,7 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class LastOnRowColumnFakeCell extends LastOnRowFakeCell {
private static class LastOnRowColCell extends LastOnRowCell {
private final byte[] fArray;
private final int foffset;
private final byte flength;
@ -1543,7 +1784,7 @@ public final class CellUtil {
private final int qoffset;
private final int qlength;
public LastOnRowColumnFakeCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
public LastOnRowColCell(byte[] rArray, int roffset, short rlength, byte[] fArray,
int foffset, byte flength, byte[] qArray, int qoffset, int qlength) {
super(rArray, roffset, rlength);
this.fArray = fArray;
@ -1586,7 +1827,7 @@ public final class CellUtil {
}
@InterfaceAudience.Private
private static class FirstOnRowDeleteFamilyCell extends FakeCell {
private static class FirstOnRowDeleteFamilyCell extends EmptyCell {
private final byte[] row;
private final byte[] fam;

View File

@ -23,6 +23,7 @@ import java.io.DataInput;
import java.io.IOException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.nio.ByteBuff;
@ -145,11 +146,15 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase
// We try to store the result in this variable so we can update stats for
// testing, but when an error happens, we log a message and return.
int block = index.rootBlockContainingKey(keyCell);
// TODO : Will be true KeyValue for now.
// When Offheap comes in we can add an else condition to work
// on the bytes in offheap
KeyValue kvKey = (KeyValue) keyCell;
return checkContains(kvKey.getBuffer(), kvKey.getKeyOffset(), kvKey.getKeyLength(), block);
// This copy will be needed. Because blooms work on the key part only.
// Atleast we now avoid multiple copies until it comes here. If we want to make this to work
// with BBs then the Hash.java APIs should also be changed to work with BBs.
if (keyCell instanceof KeyValue) {
return checkContains(((KeyValue) keyCell).getBuffer(), ((KeyValue) keyCell).getKeyOffset(),
((KeyValue) keyCell).getKeyLength(), block);
}
byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(keyCell);
return checkContains(key, 0, key.length, block);
}
public boolean supportsAutoLoading() {

View File

@ -930,6 +930,7 @@ public class StoreFile {
// merge(row, qualifier)
// TODO: could save one buffer copy in case of compound Bloom
// filters when this involves creating a KeyValue
// TODO : Handle while writes also
bloomKeyKV = KeyValueUtil.createFirstOnRow(cell.getRowArray(), cell.getRowOffset(),
cell.getRowLength(),
HConstants.EMPTY_BYTE_ARRAY, 0, 0, cell.getQualifierArray(),
@ -1219,8 +1220,8 @@ public class StoreFile {
* checks Bloom filters for single-row or single-row-column scans. Bloom
* filter checking for multi-gets is implemented as part of the store
* scanner system (see {@link StoreFileScanner#seekExactly}) and uses
* the lower-level API {@link #passesGeneralBloomFilter(byte[], int, int, byte[],
* int, int)}.
* the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
* and {@link #passesGeneralRowColBloomFilter(Cell)}.
*
* @param scan the scan specification. Used to determine the row, and to
* check whether this is a single-row ("get") scan.
@ -1241,13 +1242,16 @@ public class StoreFile {
byte[] row = scan.getStartRow();
switch (this.bloomFilterType) {
case ROW:
return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
return passesGeneralRowBloomFilter(row, 0, row.length);
case ROWCOL:
if (columns != null && columns.size() == 1) {
byte[] column = columns.first();
return passesGeneralBloomFilter(row, 0, row.length, column, 0,
column.length);
// create the required fake key
Cell kvKey = KeyValueUtil.createFirstOnRow(row, 0, row.length,
HConstants.EMPTY_BYTE_ARRAY, 0, 0, column, 0,
column.length);
return passesGeneralRowColBloomFilter(kvKey);
}
// For multi-column queries the Bloom filter is checked from the
@ -1295,15 +1299,9 @@ public class StoreFile {
* @param row
* @param rowOffset
* @param rowLen
* @param col
* @param colOffset
* @param colLen
* @return True if passes
*/
public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
int rowLen, byte[] col, int colOffset, int colLen) {
// Cache Bloom filter as a local variable in case it is set to null by
// another thread on an IO error.
public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
return true;
@ -1311,31 +1309,39 @@ public class StoreFile {
// Used in ROW bloom
byte[] key = null;
// Used in ROW_COL bloom
KeyValue kvKey = null;
switch (bloomFilterType) {
case ROW:
if (col != null) {
throw new RuntimeException("Row-only Bloom filter called with " +
"column specified");
}
if (rowOffset != 0 || rowLen != row.length) {
throw new AssertionError("For row-only Bloom filters the row "
+ "must occupy the whole array");
}
key = row;
break;
case ROWCOL:
kvKey = KeyValueUtil.createFirstOnRow(row, rowOffset, rowLen,
HConstants.EMPTY_BYTE_ARRAY, 0, 0, col, colOffset,
colLen);
break;
default:
return true;
if (rowOffset != 0 || rowLen != row.length) {
throw new AssertionError(
"For row-only Bloom filters the row " + "must occupy the whole array");
}
key = row;
return checkGeneralBloomFilter(key, null, bloomFilter);
}
/**
* A method for checking Bloom filters. Called directly from
* StoreFileScanner in case of a multi-column query.
*
* @param cell
* the cell to check if present in BloomFilter
* @return True if passes
*/
public boolean passesGeneralRowColBloomFilter(Cell cell) {
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
return true;
}
// Used in ROW_COL bloom
Cell kvKey = null;
// Already if the incoming key is a fake rowcol key then use it as it is
if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
kvKey = cell;
} else {
kvKey = CellUtil.createFirstOnRowCol(cell);
}
return checkGeneralBloomFilter(null, kvKey, bloomFilter);
}
private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
// Empty file
if (reader.getTrailer().getEntryCount() == 0)
return false;
@ -1374,8 +1380,7 @@ public class StoreFile {
// columns, a file might be skipped if using row+col Bloom filter.
// In order to ensure this file is included an additional check is
// required looking only for a row bloom.
KeyValue rowBloomKey = KeyValueUtil.createFirstOnRow(row, rowOffset, rowLen,
HConstants.EMPTY_BYTE_ARRAY, 0, 0, HConstants.EMPTY_BYTE_ARRAY, 0, 0);
Cell rowBloomKey = CellUtil.createFirstOnRow(kvKey);
// hbase:meta does not have blooms. So we need not have special interpretation
// of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
if (keyIsAfterLast

View File

@ -315,9 +315,7 @@ public class StoreFileScanner implements KeyValueScanner {
if (useBloom) {
// check ROWCOL Bloom filter first.
if (reader.getBloomFilterType() == BloomType.ROWCOL) {
haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(),
kv.getRowOffset(), kv.getRowLength(), kv.getQualifierArray(),
kv.getQualifierOffset(), kv.getQualifierLength());
haveToSeek = reader.passesGeneralRowColBloomFilter(kv);
} else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
((CellUtil.isDeleteFamily(kv) || CellUtil.isDeleteFamilyVersion(kv)))) {
// if there is no such delete family kv in the store file,