HBASE-4532 Avoid top row seek by dedicated bloom filter for delete family bloom filter

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1188268 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Nicolas Spiegelberg 2011-10-24 18:10:37 +00:00
parent e495e04c78
commit 9431c3b55c
17 changed files with 688 additions and 202 deletions

View File

@ -1764,6 +1764,21 @@ public class KeyValue implements Writable, HeapSize {
return new KeyValue(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
}
/**
* Create a Delete Family KeyValue for the specified row and family that would
* be smaller than all other possible Delete Family KeyValues that have the
* same row and family.
* Used for seeking.
* @param row - row key (arbitrary byte array)
* @param family - family name
* @return First Delete Family possible key on passed <code>row</code>.
*/
public static KeyValue createFirstDeleteFamilyOnRow(final byte [] row,
final byte [] family) {
return new KeyValue(row, family, null, HConstants.LATEST_TIMESTAMP,
Type.DeleteFamily);
}
/**
* @param row - row key (arbitrary byte array)
* @param f - family name

View File

@ -61,8 +61,11 @@ public enum BlockType {
/** File info, version 2 */
FILE_INFO("FILEINF2", BlockCategory.META),
/** Bloom filter metadata, version 2 */
BLOOM_META("BLMFMET2", BlockCategory.BLOOM),
/** General Bloom filter metadata, version 2 */
GENERAL_BLOOM_META("BLMFMET2", BlockCategory.BLOOM),
/** Delete Family Bloom filter metadata, version 2 */
DELETE_FAMILY_BLOOM_META("DFBLMET2", BlockCategory.BLOOM),
// Trailer

View File

@ -147,6 +147,9 @@ public class HFile {
static volatile AtomicLong writeOps = new AtomicLong();
static volatile AtomicLong writeTimeNano = new AtomicLong();
// for test purpose
public static volatile AtomicLong dataBlockReadCnt = new AtomicLong(0);
public static final long getReadOps() {
return readOps.getAndSet(0);
}
@ -187,11 +190,17 @@ public class HFile {
void addInlineBlockWriter(InlineBlockWriter bloomWriter);
/**
* Store Bloom filter in the file. This does not deal with Bloom filter
* Store general Bloom filter in the file. This does not deal with Bloom filter
* internals but is necessary, since Bloom filters are stored differently
* in HFile version 1 and version 2.
*/
void addBloomFilter(BloomFilterWriter bfw);
void addGeneralBloomFilter(BloomFilterWriter bfw);
/**
* Store delete family Bloom filter in the file, which is only supported in
* HFile V2.
*/
void addDeleteFamilyBloomFilter(BloomFilterWriter bfw) throws IOException;
}
/**
@ -318,10 +327,18 @@ public class HFile {
Compression.Algorithm getCompressionAlgorithm();
/**
* Retrieves Bloom filter metadata as appropriate for each {@link HFile}
* version. Knows nothing about how that metadata is structured.
* Retrieves general Bloom filter metadata as appropriate for each
* {@link HFile} version.
* Knows nothing about how that metadata is structured.
*/
DataInput getBloomFilterMetadata() throws IOException;
DataInput getGeneralBloomFilterMetadata() throws IOException;
/**
* Retrieves delete family Bloom filter metadata as appropriate for each
* {@link HFile} version.
* Knows nothing about how that metadata is structured.
*/
DataInput getDeleteBloomFilterMetadata() throws IOException;
Path getPath();

View File

@ -1455,7 +1455,6 @@ public class HFileBlock implements Cacheable, HFileBlockInfo {
}
}
}
b.offset = offset;
return b;
}

View File

@ -304,8 +304,8 @@ public class HFilePrettyPrinter {
System.out.println("Mid-key: " + Bytes.toStringBinary(reader.midkey()));
// Printing bloom information
DataInput bloomMeta = reader.getBloomFilterMetadata();
// Printing general bloom information
DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
BloomFilter bloomFilter = null;
if (bloomMeta != null)
bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
@ -317,6 +317,21 @@ public class HFilePrettyPrinter {
} else {
System.out.println(FOUR_SPACES + "Not present");
}
// Printing delete bloom information
bloomMeta = reader.getDeleteBloomFilterMetadata();
bloomFilter = null;
if (bloomMeta != null)
bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
System.out.println("Delete Family Bloom filter:");
if (bloomFilter != null) {
System.out.println(FOUR_SPACES
+ bloomFilter.toString().replaceAll(ByteBloomFilter.STATS_RECORD_SEP,
"\n" + FOUR_SPACES));
} else {
System.out.println(FOUR_SPACES + "Not present");
}
}
private static class LongStats {

View File

@ -682,7 +682,7 @@ public class HFileReaderV1 extends AbstractHFileReader {
}
@Override
public DataInput getBloomFilterMetadata() throws IOException {
public DataInput getGeneralBloomFilterMetadata() throws IOException {
ByteBuffer buf = getMetaBlock(HFileWriterV1.BLOOM_FILTER_META_KEY, false);
if (buf == null)
return null;
@ -691,6 +691,11 @@ public class HFileReaderV1 extends AbstractHFileReader {
return new DataInputStream(bais);
}
@Override
public DataInput getDeleteBloomFilterMetadata() throws IOException {
return null;
}
@Override
public boolean isFileInfoLoaded() {
return fileInfoLoaded;

View File

@ -259,6 +259,8 @@ public class HFileReaderV2 extends AbstractHFileReader {
HRegion.incrNumericMetric(this.fsBlockReadCacheHitCntMetric, 1);
}
if (cachedBlock.getBlockType() == BlockType.DATA)
HFile.dataBlockReadCnt.incrementAndGet();
return cachedBlock;
}
// Carry on, please load.
@ -285,6 +287,9 @@ public class HFileReaderV2 extends AbstractHFileReader {
cacheConf.isInMemory());
}
if (dataBlock.getBlockType() == BlockType.DATA)
HFile.dataBlockReadCnt.incrementAndGet();
return dataBlock;
} finally {
offsetLock.releaseLockEntry(lockEntry);
@ -729,9 +734,25 @@ public class HFileReaderV2 extends AbstractHFileReader {
* ownership of the buffer.
*/
@Override
public DataInput getBloomFilterMetadata() throws IOException {
public DataInput getGeneralBloomFilterMetadata() throws IOException {
return this.getBloomFilterMetadata(BlockType.GENERAL_BLOOM_META);
}
@Override
public DataInput getDeleteBloomFilterMetadata() throws IOException {
return this.getBloomFilterMetadata(BlockType.DELETE_FAMILY_BLOOM_META);
}
private DataInput getBloomFilterMetadata(BlockType blockType)
throws IOException {
if (blockType != BlockType.GENERAL_BLOOM_META &&
blockType != BlockType.DELETE_FAMILY_BLOOM_META) {
throw new RuntimeException("Block Type: " + blockType.toString() +
" is not supported") ;
}
for (HFileBlock b : loadOnOpenBlocks)
if (b.getBlockType() == BlockType.BLOOM_META)
if (b.getBlockType() == blockType)
return b.getByteStream();
return null;
}

View File

@ -441,11 +441,11 @@ public class HFileWriterV1 extends AbstractHFileWriter {
}
/**
* Version 1 Bloom filters are stored in two meta blocks with two different
* Version 1 general Bloom filters are stored in two meta blocks with two different
* keys.
*/
@Override
public void addBloomFilter(BloomFilterWriter bfw) {
public void addGeneralBloomFilter(BloomFilterWriter bfw) {
appendMetaBlock(BLOOM_FILTER_META_KEY,
bfw.getMetaWriter());
Writable dataWriter = bfw.getDataWriter();
@ -454,6 +454,12 @@ public class HFileWriterV1 extends AbstractHFileWriter {
}
}
@Override
public void addDeleteFamilyBloomFilter(BloomFilterWriter bfw)
throws IOException {
throw new IOException("Delete Bloom filter is not supported in HFile V1");
}
/**
* Write out the index in the version 1 format. This conforms to the legacy
* version 1 format, but can still be read by

View File

@ -443,14 +443,29 @@ public class HFileWriterV2 extends AbstractHFileWriter {
}
@Override
public void addBloomFilter(final BloomFilterWriter bfw) {
public void addGeneralBloomFilter(final BloomFilterWriter bfw) {
this.addBloomFilter(bfw, BlockType.GENERAL_BLOOM_META);
}
@Override
public void addDeleteFamilyBloomFilter(final BloomFilterWriter bfw) {
this.addBloomFilter(bfw, BlockType.DELETE_FAMILY_BLOOM_META);
}
private void addBloomFilter(final BloomFilterWriter bfw,
final BlockType blockType) {
if (bfw.getKeyCount() <= 0)
return;
if (blockType != BlockType.GENERAL_BLOOM_META &&
blockType != BlockType.DELETE_FAMILY_BLOOM_META) {
throw new RuntimeException("Block Type: " + blockType.toString() +
"is not supported");
}
additionalLoadOnOpenData.add(new BlockWritable() {
@Override
public BlockType getBlockType() {
return BlockType.BLOOM_META;
return blockType;
}
@Override

View File

@ -92,6 +92,14 @@ public class ScanQueryMatcher {
*/
private final long earliestPutTs;
/**
* This variable shows whether there is an null column in the query. There
* always exists a null column in the wildcard column query.
* There maybe exists a null column in the explicit column query based on the
* first column.
* */
private boolean hasNullColumn = true;
/**
* Construct a QueryMatcher for a scan
* @param scan
@ -107,7 +115,8 @@ public class ScanQueryMatcher {
this.rowComparator = scanInfo.getComparator().getRawComparator();
this.deletes = new ScanDeleteTracker();
this.stopRow = scan.getStopRow();
this.startKey = KeyValue.createFirstOnRow(scan.getStartRow(), scanInfo.getFamily(), null);
this.startKey = KeyValue.createFirstDeleteFamilyOnRow(scan.getStartRow(),
scanInfo.getFamily());
this.filter = scan.getFilter();
this.earliestPutTs = earliestPutTs;
@ -122,9 +131,15 @@ public class ScanQueryMatcher {
int maxVersions = Math.min(scan.getMaxVersions(), scanInfo.getMaxVersions());
// Single branch to deal with two types of reads (columns vs all in family)
if (columns == null || columns.size() == 0) {
// there is always a null column in the wildcard column query.
hasNullColumn = true;
// use a specialized scan for wildcard column tracker.
this.columns = new ScanWildcardColumnTracker(scanInfo.getMinVersions(), maxVersions, scanInfo.getTtl());
} else {
// whether there is null column in the explicit column query
hasNullColumn = (columns.first().length == 0);
// We can share the ExplicitColumnTracker, diff is we reset
// between rows, not between storefiles.
this.columns = new ExplicitColumnTracker(columns, scanInfo.getMinVersions(), maxVersions,
@ -141,6 +156,14 @@ public class ScanQueryMatcher {
HConstants.LATEST_TIMESTAMP);
}
/**
*
* @return whether there is an null column in the query
*/
public boolean hasNullColumnInQuery() {
return hasNullColumn;
}
/**
* Determines if the caller should do one of several things:
* - seek/skip to the next row (MatchCode.SEEK_NEXT_ROW)

View File

@ -48,6 +48,8 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HalfStoreFileReader;
import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
@ -111,6 +113,10 @@ public class StoreFile {
static final byte[] BLOOM_FILTER_TYPE_KEY =
Bytes.toBytes("BLOOM_FILTER_TYPE");
/** Delete Family Count in FileInfo */
public static final byte[] DELETE_FAMILY_COUNT =
Bytes.toBytes("DELETE_FAMILY_COUNT");
/** Last Bloom filter key in FileInfo */
private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
@ -210,7 +216,7 @@ public class StoreFile {
this.referencePath = getReferredToFile(this.path);
}
if (BloomFilterFactory.isBloomEnabled(conf)) {
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
this.cfBloomType = cfBloomType;
} else {
LOG.info("Ignoring bloom filter check for file " + path + ": " +
@ -483,7 +489,7 @@ public class StoreFile {
BloomType hfileBloomType = reader.getBloomFilterType();
if (cfBloomType != BloomType.NONE) {
reader.loadBloomfilter();
reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
if (hfileBloomType != cfBloomType) {
LOG.info("HFile Bloom filter type for "
+ reader.getHFileReader().getName() + ": " + hfileBloomType
@ -495,6 +501,9 @@ public class StoreFile {
+ reader.getHFileReader().getName());
}
// load delete family bloom filter
reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
try {
byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
if (timerangeBytes != null) {
@ -644,7 +653,7 @@ public class StoreFile {
fs.mkdirs(dir);
}
Path path = getUniqueFile(fs, dir);
if (!BloomFilterFactory.isBloomEnabled(conf)) {
if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
bloomType = BloomType.NONE;
}
@ -735,13 +744,16 @@ public class StoreFile {
* local because it is an implementation detail of the HBase regionserver.
*/
public static class Writer {
private final BloomFilterWriter bloomFilterWriter;
private final BloomFilterWriter generalBloomFilterWriter;
private final BloomFilterWriter deleteFamilyBloomFilterWriter;
private final BloomType bloomType;
private byte[] lastBloomKey;
private int lastBloomKeyOffset, lastBloomKeyLen;
private KVComparator kvComparator;
private KeyValue lastKv = null;
private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
private KeyValue lastDeleteFamilyKV = null;
private long deleteFamilyCnt = 0;
TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
/* isTimeRangeTrackerSet keeps track if the timeRange has already been set
@ -777,16 +789,32 @@ public class StoreFile {
this.kvComparator = comparator;
bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf, cacheConf,
bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
if (bloomFilterWriter != null) {
generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
conf, cacheConf, bloomType,
(int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
if (generalBloomFilterWriter != null) {
this.bloomType = bloomType;
LOG.info("Bloom filter type for " + path + ": " + this.bloomType +
", "+ bloomFilterWriter.getClass().getSimpleName());
LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
+ generalBloomFilterWriter.getClass().getSimpleName());
} else {
// Not using Bloom filters.
this.bloomType = BloomType.NONE;
}
// initialize delete family Bloom filter when there is NO RowCol Bloom
// filter
if (this.bloomType != BloomType.ROWCOL) {
this.deleteFamilyBloomFilterWriter = BloomFilterFactory
.createDeleteBloomAtWrite(conf, cacheConf,
(int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
} else {
deleteFamilyBloomFilterWriter = null;
}
if (deleteFamilyBloomFilterWriter != null) {
LOG.info("Delete Family Bloom filter type for " + path + ": "
+ deleteFamilyBloomFilterWriter.getClass().getSimpleName());
}
}
/**
@ -838,8 +866,8 @@ public class StoreFile {
}
}
public void append(final KeyValue kv) throws IOException {
if (this.bloomFilterWriter != null) {
private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
if (this.generalBloomFilterWriter != null) {
// only add to the bloom filter on a new, unique key
boolean newKey = true;
if (this.lastKv != null) {
@ -879,7 +907,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
bloomKey = bloomFilterWriter.createBloomKey(kv.getBuffer(),
bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getQualifierOffset(), kv.getQualifierLength());
bloomKeyOffset = 0;
@ -889,9 +917,9 @@ public class StoreFile {
throw new IOException("Invalid Bloom filter type: " + bloomType +
" (ROW or ROWCOL expected)");
}
bloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
if (lastBloomKey != null
&& bloomFilterWriter.getComparator().compare(bloomKey,
&& generalBloomFilterWriter.getComparator().compare(bloomKey,
bloomKeyOffset, bloomKeyLen, lastBloomKey,
lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
throw new IOException("Non-increasing Bloom keys: "
@ -906,6 +934,32 @@ public class StoreFile {
this.lastKv = kv;
}
}
}
private void appendDeleteFamilyBloomFilter(final KeyValue kv)
throws IOException {
if (!kv.isDeleteFamily()) {
return;
}
// increase the number of delete family in the store file
deleteFamilyCnt++;
if (null != this.deleteFamilyBloomFilterWriter) {
boolean newKey = true;
if (lastDeleteFamilyKV != null) {
newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
}
if (newKey) {
this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
kv.getRowOffset(), kv.getRowLength());
this.lastDeleteFamilyKV = kv;
}
}
}
public void append(final KeyValue kv) throws IOException {
appendGeneralBloomfilter(kv);
appendDeleteFamilyBloomFilter(kv);
writer.append(kv);
trackTimestamps(kv);
}
@ -914,25 +968,33 @@ public class StoreFile {
return this.writer.getPath();
}
boolean hasBloom() {
return this.bloomFilterWriter != null;
boolean hasGeneralBloom() {
return this.generalBloomFilterWriter != null;
}
/**
* For unit testing only.
*
* @return the Bloom filter used by this writer.
*/
BloomFilterWriter getBloomWriter() {
return bloomFilterWriter;
BloomFilterWriter getGeneralBloomWriter() {
return generalBloomFilterWriter;
}
public void close() throws IOException {
// Make sure we wrote something to the Bloom filter before adding it.
boolean haveBloom = bloomFilterWriter != null &&
bloomFilterWriter.getKeyCount() > 0;
private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
if (haveBloom) {
bloomFilterWriter.compactBloom();
writer.addBloomFilter(bloomFilterWriter);
bfw.compactBloom();
}
return haveBloom;
}
private boolean closeGeneralBloomFilter() throws IOException {
boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
// add the general Bloom filter writer and append file info
if (hasGeneralBloom) {
writer.addGeneralBloomFilter(generalBloomFilterWriter);
writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
Bytes.toBytes(bloomType.toString()));
if (lastBloomKey != null) {
@ -941,15 +1003,37 @@ public class StoreFile {
+ lastBloomKeyLen));
}
}
return hasGeneralBloom;
}
private boolean closeDeleteFamilyBloomFilter() throws IOException {
boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
// add the delete family Bloom filter writer
if (hasDeleteFamilyBloom) {
writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
}
// append file info about the number of delete family kvs
// even if there is no delete family Bloom.
writer.appendFileInfo(DELETE_FAMILY_COUNT,
Bytes.toBytes(this.deleteFamilyCnt));
return hasDeleteFamilyBloom;
}
public void close() throws IOException {
boolean hasGeneralBloom = this.closeGeneralBloomFilter();
boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
writer.close();
// Log final Bloom filter statistics. This needs to be done after close()
// because compound Bloom filters might be finalized as part of closing.
if (haveBloom && bloomFilterWriter.getMaxKeys() > 0) {
StoreFile.LOG.info("Bloom added to HFile ("
+ getPath() + "): " +
bloomFilterWriter.toString().replace("\n", "; "));
}
StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
+ (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
+ " was added to HFile (" + getPath() + ") ");
}
public void appendFileInfo(byte[] key, byte[] value) throws IOException {
@ -963,15 +1047,16 @@ public class StoreFile {
public static class Reader {
static final Log LOG = LogFactory.getLog(Reader.class.getName());
protected BloomFilter bloomFilter = null;
protected BloomFilter generalBloomFilter = null;
protected BloomFilter deleteFamilyBloomFilter = null;
protected BloomType bloomFilterType;
private final HFile.Reader reader;
protected TimeRangeTracker timeRangeTracker = null;
protected long sequenceID = -1;
private final String bloomAccessedMetric;
private final String bloomSkippedMetric;
private byte[] lastBloomKey;
private long deleteFamilyCnt = -1;
public Reader(FileSystem fs, Path path, CacheConfig cacheConf)
throws IOException {
@ -1087,7 +1172,7 @@ 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 #passesBloomFilter(byte[], int, int, byte[],
* the lower-level API {@link #passesGeneralBloomFilter(byte[], int, int, byte[],
* int, int)}.
*
* @param scan the scan specification. Used to determine the row, and to
@ -1109,12 +1194,12 @@ public class StoreFile {
byte[] row = scan.getStartRow();
switch (this.bloomFilterType) {
case ROW:
return passesBloomFilter(row, 0, row.length, null, 0, 0);
return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
case ROWCOL:
if (columns != null && columns.size() == 1) {
byte[] column = columns.first();
return passesBloomFilter(row, 0, row.length, column, 0,
return passesGeneralBloomFilter(row, 0, row.length, column, 0,
column.length);
}
@ -1127,6 +1212,35 @@ public class StoreFile {
}
}
public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
int rowLen) {
// Cache Bloom filter as a local variable in case it is set to null by
// another thread on an IO error.
BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
// Empty file or there is no delete family at all
if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
return false;
}
if (bloomFilter == null) {
return true;
}
try {
if (!bloomFilter.supportsAutoLoading()) {
return true;
}
return bloomFilter.contains(row, rowOffset, rowLen, null);
} catch (IllegalArgumentException e) {
LOG.error("Bad Delete Family bloom filter data -- proceeding without",
e);
setDeleteFamilyBloomFilterFaulty();
}
return true;
}
/**
* A method for checking Bloom filters. Called directly from
* {@link StoreFileScanner} in case of a multi-column query.
@ -1139,9 +1253,9 @@ public class StoreFile {
* @param colLen
* @return
*/
public boolean passesBloomFilter(byte[] row, int rowOffset, int rowLen,
byte[] col, int colOffset, int colLen) {
if (bloomFilter == null)
public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
int rowLen, byte[] col, int colOffset, int colLen) {
if (generalBloomFilter == null)
return true;
byte[] key;
@ -1159,7 +1273,7 @@ public class StoreFile {
break;
case ROWCOL:
key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
key = generalBloomFilter.createBloomKey(row, rowOffset, rowLen, col,
colOffset, colLen);
break;
@ -1169,13 +1283,13 @@ public class StoreFile {
// Cache Bloom filter as a local variable in case it is set to null by
// another thread on an IO error.
BloomFilter bloomFilter = this.bloomFilter;
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
return true;
}
// Empty file?
// Empty file
if (reader.getTrailer().getEntryCount() == 0)
return false;
@ -1214,13 +1328,13 @@ public class StoreFile {
exists = false;
} else {
exists =
this.bloomFilter.contains(key, 0, key.length, bloom) ||
this.bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
bloomFilter.contains(key, 0, key.length, bloom) ||
bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
bloom);
}
} else {
exists = !keyIsAfterLast
&& this.bloomFilter.contains(key, 0, key.length, bloom);
&& bloomFilter.contains(key, 0, key.length, bloom);
}
if (exists)
@ -1232,10 +1346,10 @@ public class StoreFile {
} catch (IOException e) {
LOG.error("Error reading bloom filter data -- proceeding without",
e);
setBloomFilterFaulty();
setGeneralBloomFilterFaulty();
} catch (IllegalArgumentException e) {
LOG.error("Bad bloom filter data -- proceeding without", e);
setBloomFilterFaulty();
setGeneralBloomFilterFaulty();
}
return true;
@ -1250,34 +1364,71 @@ public class StoreFile {
}
lastBloomKey = fi.get(LAST_BLOOM_KEY);
byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
if (cnt != null) {
deleteFamilyCnt = Bytes.toLong(cnt);
}
return fi;
}
public void loadBloomfilter() {
if (this.bloomFilter != null) {
return; // already loaded
}
this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
}
private void loadBloomfilter(BlockType blockType) {
try {
DataInput bloomMeta = reader.getBloomFilterMetadata();
if (bloomMeta != null) {
if (bloomFilterType == BloomType.NONE) {
throw new IOException(
"valid bloom filter type not found in FileInfo");
}
if (blockType == BlockType.GENERAL_BLOOM_META) {
if (this.generalBloomFilter != null)
return; // Bloom has been loaded
bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
LOG.info("Loaded " + bloomFilterType + " " +
bloomFilter.getClass().getSimpleName() + " metadata for " +
reader.getName());
DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
if (bloomMeta != null) {
// sanity check for NONE Bloom filter
if (bloomFilterType == BloomType.NONE) {
throw new IOException(
"valid bloom filter type not found in FileInfo");
} else {
generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
reader);
LOG.info("Loaded " + bloomFilterType.toString() + " ("
+ generalBloomFilter.getClass().getSimpleName()
+ ") metadata for " + reader.getName());
}
}
} else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
if (this.deleteFamilyBloomFilter != null)
return; // Bloom has been loaded
DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
if (bloomMeta != null) {
deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
bloomMeta, reader);
LOG.info("Loaded Delete Family Bloom ("
+ deleteFamilyBloomFilter.getClass().getSimpleName()
+ ") metadata for " + reader.getName());
}
} else {
throw new RuntimeException("Block Type: " + blockType.toString()
+ "is not supported for Bloom filter");
}
} catch (IOException e) {
LOG.error("Error reading bloom filter meta -- proceeding without", e);
this.bloomFilter = null;
LOG.error("Error reading bloom filter meta for " + blockType
+ " -- proceeding without", e);
setBloomFilterFaulty(blockType);
} catch (IllegalArgumentException e) {
LOG.error("Bad bloom filter meta -- proceeding without", e);
this.bloomFilter = null;
LOG.error("Bad bloom filter meta " + blockType
+ " -- proceeding without", e);
setBloomFilterFaulty(blockType);
}
}
private void setBloomFilterFaulty(BlockType blockType) {
if (blockType == BlockType.GENERAL_BLOOM_META) {
setGeneralBloomFilterFaulty();
} else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
setDeleteFamilyBloomFilterFaulty();
}
}
@ -1289,12 +1440,16 @@ public class StoreFile {
* @return an estimate of the number of Bloom filter entries in this file
*/
public long getFilterEntries() {
return bloomFilter != null ? bloomFilter.getKeyCount()
return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
: reader.getEntries();
}
public void setBloomFilterFaulty() {
bloomFilter = null;
public void setGeneralBloomFilterFaulty() {
generalBloomFilter = null;
}
public void setDeleteFamilyBloomFilterFaulty() {
this.deleteFamilyBloomFilter = null;
}
public byte[] getLastKey() {
@ -1317,6 +1472,10 @@ public class StoreFile {
return reader.getEntries();
}
public long getDeleteFamilyCnt() {
return deleteFamilyCnt;
}
public byte[] getFirstKey() {
return reader.getFirstKey();
}
@ -1341,8 +1500,8 @@ public class StoreFile {
this.sequenceID = sequenceID;
}
BloomFilter getBloomFilter() {
return bloomFilter;
BloomFilter getGeneralBloomFilter() {
return generalBloomFilter;
}
long getUncompressedDataIndexSize() {
@ -1350,9 +1509,9 @@ public class StoreFile {
}
public long getTotalBloomSize() {
if (bloomFilter == null)
if (generalBloomFilter == null)
return 0;
return bloomFilter.getByteSize();
return generalBloomFilter.getByteSize();
}
public int getHFileVersion() {
@ -1364,7 +1523,8 @@ public class StoreFile {
}
void disableBloomFilterForTesting() {
bloomFilter = null;
generalBloomFilter = null;
this.deleteFamilyBloomFilter = null;
}
public long getMaxTimestamp() {

View File

@ -24,8 +24,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
@ -57,6 +59,8 @@ class StoreFileScanner implements KeyValueScanner {
private static final AtomicLong seekCount = new AtomicLong();
private ScanQueryMatcher matcher;
/**
* Implements a {@link KeyValueScanner} on top of the specified {@link HFileScanner}
* @param hfs HFile scanner
@ -79,19 +83,31 @@ class StoreFileScanner implements KeyValueScanner {
}
/**
* Return an array of scanners corresponding to the given
* set of store files.
* Return an array of scanners corresponding to the given set of store files.
*/
public static List<StoreFileScanner> getScannersForStoreFiles(
Collection<StoreFile> files,
boolean cacheBlocks,
boolean usePread,
Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
boolean isCompaction) throws IOException {
List<StoreFileScanner> scanners =
new ArrayList<StoreFileScanner>(files.size());
return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction,
null);
}
/**
* Return an array of scanners corresponding to the given set of store files,
* And set the ScanQueryMatcher for each store file scanner for further
* optimization
*/
public static List<StoreFileScanner> getScannersForStoreFiles(
Collection<StoreFile> files, boolean cacheBlocks, boolean usePread,
boolean isCompaction, ScanQueryMatcher matcher) throws IOException {
List<StoreFileScanner> scanners = new ArrayList<StoreFileScanner>(
files.size());
for (StoreFile file : files) {
StoreFile.Reader r = file.createReader();
scanners.add(r.getStoreFileScanner(cacheBlocks, usePread, isCompaction));
StoreFileScanner scanner = r.getStoreFileScanner(cacheBlocks, usePread,
isCompaction);
scanner.setScanQueryMatcher(matcher);
scanners.add(scanner);
}
return scanners;
}
@ -140,6 +156,7 @@ class StoreFileScanner implements KeyValueScanner {
public boolean reseek(KeyValue key) throws IOException {
seekCount.incrementAndGet();
System.out.println("Reseek " + key + " data block " + HFile.dataBlockReadCnt);
try {
try {
if (!reseekAtOrAfter(hfs, key)) {
@ -223,16 +240,24 @@ class StoreFileScanner implements KeyValueScanner {
@Override
public boolean requestSeek(KeyValue kv, boolean forward, boolean useBloom)
throws IOException {
if (reader.getBloomFilterType() != StoreFile.BloomType.ROWCOL ||
kv.getFamilyLength() == 0) {
if (kv.getFamilyLength() == 0) {
useBloom = false;
}
boolean haveToSeek = true;
if (useBloom) {
haveToSeek = reader.passesBloomFilter(kv.getBuffer(),
kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getQualifierOffset(), kv.getQualifierLength());
// check ROWCOL Bloom filter first.
if (reader.getBloomFilterType() == StoreFile.BloomType.ROWCOL) {
haveToSeek = reader.passesGeneralBloomFilter(kv.getBuffer(),
kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
kv.getQualifierOffset(), kv.getQualifierLength());
} else if (this.matcher != null && !matcher.hasNullColumnInQuery() &&
kv.isDeleteFamily()) {
// if there is no such delete family kv in the store file,
// then no need to seek.
haveToSeek = reader.passesDeleteFamilyBloomFilter(kv.getBuffer(),
kv.getRowOffset(), kv.getRowLength());
}
}
delayedReseek = forward;
@ -296,6 +321,10 @@ class StoreFileScanner implements KeyValueScanner {
}
}
public void setScanQueryMatcher(ScanQueryMatcher matcher) {
this.matcher = matcher;
}
// Test methods
static final long getSeekCount() {

View File

@ -196,8 +196,9 @@ class StoreScanner extends NonLazyKeyValueScanner
List<KeyValueScanner> scanners = new LinkedList<KeyValueScanner>();
// First the store file scanners
if (memOnly == false) {
List<StoreFileScanner> sfScanners = StoreFileScanner
.getScannersForStoreFiles(store.getStorefiles(), cacheBlocks, isGet);
List<StoreFileScanner> sfScanners = StoreFileScanner
.getScannersForStoreFiles(store.getStorefiles(), cacheBlocks,
isGet, false, this.matcher);
// include only those scan files which pass all filters
for (StoreFileScanner sfs : sfScanners) {

View File

@ -69,6 +69,10 @@ public final class BloomFilterFactory {
public static final String IO_STOREFILE_BLOOM_ENABLED =
"io.storefile.bloom.enabled";
/** Master switch to enable Delete Family Bloom filters */
public static final String IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED =
"io.storefile.delete.family.bloom.enabled";
/**
* Target Bloom block size. Bloom filter blocks of approximately this size
* are interleaved with data blocks.
@ -112,18 +116,48 @@ public final class BloomFilterFactory {
}
/**
* @return true if Bloom filters are enabled in the given configuration
* @return true if general Bloom (Row or RowCol) filters are enabled in the
* given configuration
*/
public static boolean isBloomEnabled(Configuration conf) {
public static boolean isGeneralBloomEnabled(Configuration conf) {
return conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true);
}
/**
* @return true if Delete Family Bloom filters are enabled in the given configuration
*/
public static boolean isDeleteFamilyBloomEnabled(Configuration conf) {
return conf.getBoolean(IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED, true);
}
/**
* @return the Bloom filter error rate in the given configuration
*/
public static float getErrorRate(Configuration conf) {
return conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
}
/**
* Creates a new Bloom filter at the time of
* @return the value for Bloom filter max fold in the given configuration
*/
public static int getMaxFold(Configuration conf) {
return conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, MAX_ALLOWED_FOLD_FACTOR);
}
/** @return the compound Bloom filter block size from the configuration */
public static int getBloomBlockSize(Configuration conf) {
return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
}
/**
* @return max key for the Bloom filter from the configuration
*/
public static int getMaxKeys(Configuration conf) {
return conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128 * 1000 * 1000);
}
/**
* Creates a new general (Row or RowCol) Bloom filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
*
* @param conf
@ -131,15 +165,14 @@ public final class BloomFilterFactory {
* @param maxKeys an estimate of the number of keys we expect to insert.
* Irrelevant if compound Bloom filters are enabled.
* @param writer the HFile writer
* @param comparator the comparator to use for compound Bloom filters. This
* has no effect if creating single-chunk version 1 Bloom filters.
* @param bloomErrorRate
* @return the new Bloom filter, or null in case Bloom filters are disabled
* or when failed to create one.
*/
public static BloomFilterWriter createBloomAtWrite(Configuration conf,
public static BloomFilterWriter createGeneralBloomAtWrite(Configuration conf,
CacheConfig cacheConf, BloomType bloomType, int maxKeys,
HFile.Writer writer) {
if (!isBloomEnabled(conf)) {
if (!isGeneralBloomEnabled(conf)) {
LOG.info("Bloom filters are disabled by configuration for "
+ writer.getPath()
+ (conf == null ? " (configuration is null)" : ""));
@ -194,8 +227,40 @@ public final class BloomFilterFactory {
return null;
}
/** @return the compound Bloom filter block size from the configuration */
public static int getBloomBlockSize(Configuration conf) {
return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
/**
* Creates a new Delete Family Bloom filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
* @param conf
* @param maxKeys an estimate of the number of keys we expect to insert.
* Irrelevant if compound Bloom filters are enabled.
* @param writer the HFile writer
* @param bloomErrorRate
* @return the new Bloom filter, or null in case Bloom filters are disabled
* or when failed to create one.
*/
public static BloomFilterWriter createDeleteBloomAtWrite(Configuration conf,
CacheConfig cacheConf, int maxKeys, HFile.Writer writer) {
if (!isDeleteFamilyBloomEnabled(conf)) {
LOG.info("Delete Bloom filters are disabled by configuration for "
+ writer.getPath()
+ (conf == null ? " (configuration is null)" : ""));
return null;
}
float err = getErrorRate(conf);
if (HFile.getFormatVersion(conf) > HFile.MIN_FORMAT_VERSION) {
int maxFold = getMaxFold(conf);
// In case of compound Bloom filters we ignore the maxKeys hint.
CompoundBloomFilterWriter bloomWriter = new CompoundBloomFilterWriter(
getBloomBlockSize(conf), err, Hash.getHashType(conf),
maxFold,
cacheConf.shouldCacheBloomsOnWrite(), Bytes.BYTES_RAWCOMPARATOR);
writer.addInlineBlockWriter(bloomWriter);
return bloomWriter;
} else {
LOG.info("Delete Family Bloom filter is not supported in HFile V1");
return null;
}
}
};

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@ -48,6 +49,7 @@ import org.junit.Test;
public class TestBlocksRead extends HBaseTestCase {
static final Log LOG = LogFactory.getLog(TestBlocksRead.class);
static final String[] BLOOM_TYPE = new String[] { "ROWCOL", "ROW", "NONE" };
private static BlockCache blockCache;
@ -79,32 +81,36 @@ public class TestBlocksRead extends HBaseTestCase {
EnvironmentEdgeManagerTestHelper.reset();
}
private void initHRegion (byte [] tableName, String callingMethod,
HBaseConfiguration conf, byte [] ... families)
throws IOException {
private void initHRegion(byte[] tableName, String callingMethod,
HBaseConfiguration conf, String family) throws IOException {
HTableDescriptor htd = new HTableDescriptor(tableName);
for(byte [] family : families) {
HColumnDescriptor familyDesc = new HColumnDescriptor(
family,
HColumnDescriptor familyDesc;
for (int i = 0; i < BLOOM_TYPE.length; i++) {
String bloomType = BLOOM_TYPE[i];
familyDesc = new HColumnDescriptor(
Bytes.toBytes(family + "_" + bloomType),
HColumnDescriptor.DEFAULT_VERSIONS,
HColumnDescriptor.DEFAULT_COMPRESSION,
HColumnDescriptor.DEFAULT_IN_MEMORY,
HColumnDescriptor.DEFAULT_BLOCKCACHE,
1, // small block size deliberate; each kv on its own block
HColumnDescriptor.DEFAULT_TTL,
HColumnDescriptor.DEFAULT_BLOOMFILTER,
HColumnDescriptor.DEFAULT_TTL, BLOOM_TYPE[i],
HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
htd.addFamily(familyDesc);
}
HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
Path path = new Path(DIR + callingMethod);
region = HRegion.createHRegion(info, path, conf, htd);
blockCache = new CacheConfig(conf).getBlockCache();
}
private void putData(byte[] cf, String row, String col, long version)
throws IOException {
putData(cf, row, col, version, version);
private void putData(String family, String row, String col, long version)
throws IOException {
for (int i = 0; i < BLOOM_TYPE.length; i++) {
putData(Bytes.toBytes(family + "_" + BLOOM_TYPE[i]), row, col, version,
version);
}
}
// generates a value to put for a row/col/version.
@ -112,9 +118,8 @@ public class TestBlocksRead extends HBaseTestCase {
return Bytes.toBytes("Value:" + row + "#" + col + "#" + version);
}
private void putData(byte[] cf, String row, String col,
long versionStart, long versionEnd)
throws IOException {
private void putData(byte[] cf, String row, String col, long versionStart,
long versionEnd) throws IOException {
byte columnBytes[] = Bytes.toBytes(col);
Put put = new Put(Bytes.toBytes(row));
@ -124,59 +129,73 @@ public class TestBlocksRead extends HBaseTestCase {
region.put(put);
}
private KeyValue[] getData(byte[] cf, String row, List<String> columns,
int expBlocks)
throws IOException {
private KeyValue[] getData(String family, String row, List<String> columns,
int expBlocks) throws IOException {
return getData(family, row, columns, expBlocks, expBlocks, expBlocks);
}
long blocksStart = getBlkAccessCount(cf);
Get get = new Get(Bytes.toBytes(row));
private KeyValue[] getData(String family, String row, List<String> columns,
int expBlocksRowCol, int expBlocksRow, int expBlocksNone)
throws IOException {
int[] expBlocks = new int[] { expBlocksRowCol, expBlocksRow, expBlocksNone };
KeyValue[] kvs = null;
for (String column : columns) {
get.addColumn(cf, Bytes.toBytes(column));
for (int i = 0; i < BLOOM_TYPE.length; i++) {
String bloomType = BLOOM_TYPE[i];
byte[] cf = Bytes.toBytes(family + "_" + bloomType);
long blocksStart = getBlkAccessCount(cf);
Get get = new Get(Bytes.toBytes(row));
for (String column : columns) {
get.addColumn(cf, Bytes.toBytes(column));
}
kvs = region.get(get, null).raw();
long blocksEnd = getBlkAccessCount(cf);
if (expBlocks[i] != -1) {
assertEquals("Blocks Read Check for Bloom: " + bloomType, expBlocks[i],
blocksEnd - blocksStart);
}
System.out.println("Blocks Read for Bloom: " + bloomType + " = "
+ (blocksEnd - blocksStart) + "Expected = " + expBlocks[i]);
}
KeyValue[] kvs = region.get(get, null).raw();
long blocksEnd = getBlkAccessCount(cf);
if (expBlocks != -1) {
assertEquals("Blocks Read Check", expBlocks, blocksEnd - blocksStart);
}
System.out.println("Blocks Read = " + (blocksEnd - blocksStart) +
"Expected = " + expBlocks);
return kvs;
}
private KeyValue[] getData(byte[] cf, String row, String column,
int expBlocks)
throws IOException {
return getData(cf, row, Arrays.asList(column), expBlocks);
private KeyValue[] getData(String family, String row, String column,
int expBlocks) throws IOException {
return getData(family, row, Arrays.asList(column), expBlocks, expBlocks,
expBlocks);
}
private void deleteFamily(byte[] cf, String row, long version)
throws IOException {
Delete del = new Delete(Bytes.toBytes(row));
del.deleteFamily(cf, version);
region.delete(del, null, true);
private KeyValue[] getData(String family, String row, String column,
int expBlocksRowCol, int expBlocksRow, int expBlocksNone)
throws IOException {
return getData(family, row, Arrays.asList(column), expBlocksRowCol,
expBlocksRow, expBlocksNone);
}
private void deleteFamily(byte[] cf, String row, String column, long version)
throws IOException {
private void deleteFamily(String family, String row, long version)
throws IOException {
Delete del = new Delete(Bytes.toBytes(row));
del.deleteColumns(cf, Bytes.toBytes(column), version);
del.deleteFamily(Bytes.toBytes(family + "_ROWCOL"), version);
del.deleteFamily(Bytes.toBytes(family + "_ROW"), version);
del.deleteFamily(Bytes.toBytes(family + "_NONE"), version);
region.delete(del, null, true);
}
private static void verifyData(KeyValue kv, String expectedRow,
String expectedCol, long expectedVersion) {
String expectedCol, long expectedVersion) {
assertEquals("RowCheck", expectedRow, Bytes.toString(kv.getRow()));
assertEquals("ColumnCheck", expectedCol, Bytes.toString(kv.getQualifier()));
assertEquals("TSCheck", expectedVersion, kv.getTimestamp());
assertEquals("ValueCheck",
Bytes.toString(genValue(expectedRow, expectedCol, expectedVersion)),
Bytes.toString(kv.getValue()));
Bytes.toString(genValue(expectedRow, expectedCol, expectedVersion)),
Bytes.toString(kv.getValue()));
}
private static long getBlkAccessCount(byte[] cf) {
return blockCache.getStats().getRequestCount();
return HFile.dataBlockReadCnt.get();
}
private static long getBlkCount() {
@ -185,17 +204,16 @@ public class TestBlocksRead extends HBaseTestCase {
/**
* Test # of blocks read for some simple seek cases.
*
* @throws Exception
*/
@Test
public void testBlocksRead() throws Exception {
byte [] TABLE = Bytes.toBytes("testBlocksRead");
byte [] FAMILY = Bytes.toBytes("cf1");
byte [][] FAMILIES = new byte[][] { FAMILY };
byte[] TABLE = Bytes.toBytes("testBlocksRead");
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILIES);
initHRegion(TABLE, getName(), conf, FAMILY);
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
@ -207,41 +225,47 @@ public class TestBlocksRead extends HBaseTestCase {
region.flushcache();
// Expected block reads: 1
// The top block has the KV we are
// interested. So only 1 seek is needed.
kvs = getData(FAMILY, "row", "col1", 1);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col1", 1);
// Expected block reads: 2
// The top block and next block has the KVs we are
// interested. So only 2 seek is needed.
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
assertEquals(2, kvs.length);
verifyData(kvs[0], "row", "col1", 1);
verifyData(kvs[1], "row", "col2", 2);
// Expected block reads: 3
// The first 2 seeks is to find out col2. [HBASE-4443]
// One additional seek for col3
// So 3 seeks are needed.
kvs = getData(FAMILY, "row", Arrays.asList("col2", "col3"), 3);
assertEquals(2, kvs.length);
verifyData(kvs[0], "row", "col2", 2);
verifyData(kvs[1], "row", "col3", 3);
// Expected block reads: 3
kvs = getData(FAMILY, "row", Arrays.asList("col5"), 3);
// Expected block reads: 2. [HBASE-4443]
kvs = getData(FAMILY, "row", Arrays.asList("col5"), 2);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col5", 5);
}
/**
* Test # of blocks read (targetted at some of the cases Lazy Seek optimizes).
*
* @throws Exception
*/
@Test
public void testLazySeekBlocksRead() throws Exception {
byte [] TABLE = Bytes.toBytes("testLazySeekBlocksRead");
byte [] FAMILY = Bytes.toBytes("cf1");
byte [][] FAMILIES = new byte[][] { FAMILY };
byte[] TABLE = Bytes.toBytes("testLazySeekBlocksRead");
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILIES);
initHRegion(TABLE, getName(), conf, FAMILY);
// File 1
putData(FAMILY, "row", "col1", 1);
@ -253,13 +277,18 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col2", 4);
region.flushcache();
// Baseline expected blocks read: 2
kvs = getData(FAMILY, "row", Arrays.asList("col1"), 2);
// Expected blocks read: 1.
// File 2's top block is also the KV we are
// interested. So only 1 seek is needed.
kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col1", 3);
// Expected block reads: 3
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 3);
// Expected blocks read: 2
// File 2's top block has the "col1" KV we are
// interested. We also need "col2" which is in a block
// of its own. So, we need that block as well.
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2"), 2);
assertEquals(2, kvs.length);
verifyData(kvs[0], "row", "col1", 3);
verifyData(kvs[1], "row", "col2", 4);
@ -268,14 +297,18 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col3", 5);
region.flushcache();
// Baseline expected blocks read: 3
kvs = getData(FAMILY, "row", "col3", 3);
// Expected blocks read: 1
// File 3's top block has the "col3" KV we are
// interested. So only 1 seek is needed.
kvs = getData(FAMILY, "row", "col3", 1);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col3", 5);
// Get a column from older file.
// Baseline expected blocks read: 3
kvs = getData(FAMILY, "row", Arrays.asList("col1"), 3);
// For ROWCOL Bloom filter: Expected blocks read: 1.
// For ROW Bloom filter: Expected blocks read: 2.
// For NONE Bloom filter: Expected blocks read: 2.
kvs = getData(FAMILY, "row", Arrays.asList("col1"), 1, 2, 2);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col1", 3);
@ -283,14 +316,16 @@ public class TestBlocksRead extends HBaseTestCase {
deleteFamily(FAMILY, "row", 6);
region.flushcache();
// Baseline expected blocks read: 4. [HBASE-4585]
kvs = getData(FAMILY, "row", "col1", 4);
// For ROWCOL Bloom filter: Expected blocks read: 2.
// For ROW Bloom filter: Expected blocks read: 3.
// For NONE Bloom filter: Expected blocks read: 3.
kvs = getData(FAMILY, "row", "col1", 2, 3, 3);
assertEquals(0, kvs.length);
kvs = getData(FAMILY, "row", "col2", 5);
kvs = getData(FAMILY, "row", "col2", 3, 4, 4);
assertEquals(0, kvs.length);
kvs = getData(FAMILY, "row", "col3", 4);
kvs = getData(FAMILY, "row", "col3", 2);
assertEquals(0, kvs.length);
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 4);
assertEquals(0, kvs.length);
// File 5: Delete
@ -304,8 +339,8 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col3", 9);
region.flushcache();
// Baseline expected blocks read: 8. [HBASE-4585]
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 8);
// Baseline expected blocks read: 8. [HBASE-4532]
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
assertEquals(0, kvs.length);
// File 7: Put back new data
@ -314,8 +349,9 @@ public class TestBlocksRead extends HBaseTestCase {
putData(FAMILY, "row", "col3", 13);
region.flushcache();
// Baseline expected blocks read: 9
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 9);
// Expected blocks read: 5. [HBASE-4585]
kvs = getData(FAMILY, "row", Arrays.asList("col1", "col2", "col3"), 5);
assertEquals(3, kvs.length);
verifyData(kvs[0], "row", "col1", 11);
verifyData(kvs[1], "row", "col2", 12);
@ -329,11 +365,10 @@ public class TestBlocksRead extends HBaseTestCase {
@Test
public void testBlocksStoredWhenCachingDisabled() throws Exception {
byte [] TABLE = Bytes.toBytes("testBlocksReadWhenCachingDisabled");
byte [] FAMILY = Bytes.toBytes("cf1");
byte [][] FAMILIES = new byte[][] { FAMILY };
String FAMILY = "cf1";
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILIES);
initHRegion(TABLE, getName(), conf, FAMILY);
putData(FAMILY, "row", "col1", 1);
putData(FAMILY, "row", "col2", 2);
@ -347,7 +382,7 @@ public class TestBlocksRead extends HBaseTestCase {
RegionScanner rs = region.getScanner(scan);
List<KeyValue> result = new ArrayList<KeyValue>(2);
rs.next(result);
assertEquals(2, result.size());
assertEquals(2 * BLOOM_TYPE.length, result.size());
rs.close();
long blocksEnd = getBlkCount();
@ -360,10 +395,33 @@ public class TestBlocksRead extends HBaseTestCase {
rs = region.getScanner(scan);
result = new ArrayList<KeyValue>(2);
rs.next(result);
assertEquals(2, result.size());
assertEquals(2 * BLOOM_TYPE.length, result.size());
rs.close();
blocksEnd = getBlkCount();
assertEquals(2, blocksEnd - blocksStart);
}
assertEquals(2 * BLOOM_TYPE.length, blocksEnd - blocksStart);
}
@Test
public void testLazySeekBlocksReadWithDelete() throws Exception {
byte[] TABLE = Bytes.toBytes("testLazySeekBlocksReadWithDelete");
String FAMILY = "cf1";
KeyValue kvs[];
HBaseConfiguration conf = getConf();
initHRegion(TABLE, getName(), conf, FAMILY);
deleteFamily(FAMILY, "row", 200);
for (int i = 0; i < 100; i++) {
putData(FAMILY, "row", "col" + i, i);
}
putData(FAMILY, "row", "col99", 201);
region.flushcache();
kvs = getData(FAMILY, "row", Arrays.asList("col0"), 2);
assertEquals(0, kvs.length);
kvs = getData(FAMILY, "row", Arrays.asList("col99"), 2);
assertEquals(1, kvs.length);
verifyData(kvs[0], "row", "col99", 201);
}
}

View File

@ -218,7 +218,7 @@ public class TestCompoundBloomFilter {
try {
String fakeLookupModeStr = ", fake lookup is " + (fakeLookupEnabled ?
"enabled" : "disabled");
CompoundBloomFilter cbf = (CompoundBloomFilter) r.getBloomFilter();
CompoundBloomFilter cbf = (CompoundBloomFilter) r.getGeneralBloomFilter();
cbf.enableTestingStats();
int numFalsePos = 0;
Random rand = new Random(EVALUATION_SEED);
@ -296,10 +296,10 @@ public class TestCompoundBloomFilter {
HBaseTestingUtility.getTestDir(), BLOCK_SIZES[t], null, null, conf,
cacheConf, bt, 0);
assertTrue(w.hasBloom());
assertTrue(w.getBloomWriter() instanceof CompoundBloomFilterWriter);
assertTrue(w.hasGeneralBloom());
assertTrue(w.getGeneralBloomWriter() instanceof CompoundBloomFilterWriter);
CompoundBloomFilterWriter cbbf =
(CompoundBloomFilterWriter) w.getBloomWriter();
(CompoundBloomFilterWriter) w.getGeneralBloomWriter();
int keyCount = 0;
KeyValue prev = null;

View File

@ -394,6 +394,60 @@ public class TestStoreFile extends HBaseTestCase {
bloomWriteRead(writer, fs);
}
public void testDeleteFamilyBloomFilter() throws Exception {
FileSystem fs = FileSystem.getLocal(conf);
conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
(float) 0.01);
conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
float err = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
0);
// write the file
Path f = new Path(ROOT_DIR, getName());
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
// add delete family
long now = System.currentTimeMillis();
for (int i = 0; i < 2000; i += 2) {
String row = String.format(localFormatter, i);
KeyValue kv = new KeyValue(row.getBytes(), "family".getBytes(),
"col".getBytes(), now, KeyValue.Type.DeleteFamily, "value".getBytes());
writer.append(kv);
}
writer.close();
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
reader.loadFileInfo();
reader.loadBloomfilter();
// check false positives rate
int falsePos = 0;
int falseNeg = 0;
for (int i = 0; i < 2000; i++) {
String row = String.format(localFormatter, i);
byte[] rowKey = Bytes.toBytes(row);
boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0,
rowKey.length);
if (i % 2 == 0) {
if (!exists)
falseNeg++;
} else {
if (exists)
falsePos++;
}
}
assertEquals(1000, reader.getDeleteFamilyCnt());
reader.close(true); // evict because we are about to delete the file
fs.delete(f, true);
assertEquals("False negatives: " + falseNeg, 0, falseNeg);
int maxFalsePos = (int) (2 * 2000 * err);
assertTrue("Too many false positives: " + falsePos + " (err=" + err
+ ", expected no more than " + maxFalsePos, falsePos <= maxFalsePos);
}
public void testBloomTypes() throws Exception {
float err = (float) 0.01;
FileSystem fs = FileSystem.getLocal(conf);
@ -440,8 +494,8 @@ public class TestStoreFile extends HBaseTestCase {
StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
reader.loadFileInfo();
reader.loadBloomfilter();
StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
assertEquals(expKeys[x], reader.bloomFilter.getKeyCount());
StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
assertEquals(expKeys[x], reader.generalBloomFilter.getKeyCount());
// check false positives rate
int falsePos = 0;
@ -491,7 +545,7 @@ public class TestStoreFile extends HBaseTestCase {
StoreFile.Writer writer = new StoreFile.Writer(fs, f,
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
assertFalse(writer.hasBloom());
assertFalse(writer.hasGeneralBloom());
writer.close();
fs.delete(f, true);
@ -515,7 +569,7 @@ public class TestStoreFile extends HBaseTestCase {
StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
Integer.MAX_VALUE);
assertFalse(writer.hasBloom());
assertFalse(writer.hasGeneralBloom());
writer.close();
fs.delete(f, true);
}