.../hadoop/hbase/io/hfile/HFileBlockIndex.java | 1 - .../hadoop/hbase/regionserver/StoreFile.java | 38 ++++++++++++---------- .../org/apache/hadoop/hbase/util/BloomFilter.java | 13 +++++++- .../apache/hadoop/hbase/util/BloomFilterBase.java | 5 +-- .../hadoop/hbase/util/BloomFilterFactory.java | 7 ---- .../apache/hadoop/hbase/util/ByteBloomFilter.java | 23 +++---------- .../hadoop/hbase/util/CompoundBloomFilter.java | 18 ++++++++-- .../hadoop/hbase/util/CompoundBloomFilterBase.java | 10 +++--- .../regionserver/TestCompoundBloomFilter.java | 8 ++--- 9 files changed, 64 insertions(+), 59 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java index e6e1fff..7707115 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlockIndex.java @@ -448,7 +448,6 @@ public class HFileBlockIndex { * Key to find */ public int rootBlockContainingKey(final Cell key) { - // Here the comparator should not be null as this happens for the root-level block int pos = Bytes.binarySearch(blockKeys, key, comparator); // pos is between -(blockKeys.length + 1) to blockKeys.length - 1, see // binarySearch's javadoc. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 1992479..61d079e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -864,7 +864,9 @@ public class StoreFile { * 1. Row = Row * 2. RowCol = Row + Qualifier */ - byte[] bloomKey; + byte[] bloomKey = null; + // Used with ROW_COL bloom + KeyValue bloomKeyKV = null; int bloomKeyOffset, bloomKeyLen; switch (bloomType) { @@ -877,9 +879,10 @@ 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 = generalBloomFilterWriter.createBloomKey(cell.getRowArray(), + bloomKeyKV = generalBloomFilterWriter.createBloomKeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()); + bloomKey = bloomKeyKV.getKey(); bloomKeyOffset = 0; bloomKeyLen = bloomKey.length; break; @@ -889,17 +892,17 @@ public class StoreFile { } generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen); if (lastBloomKey != null) { - boolean res = false; + int res = 0; // 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 (bloomType == BloomType.ROW) { res = Bytes.BYTES_RAWCOMPARATOR.compare(bloomKey, bloomKeyOffset, bloomKeyLen, - lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen) <= 0; + lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen); } else { - res = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, bloomKey, - bloomKeyOffset, bloomKeyLen) >= 0); + // TODO : Caching of kv components becomes important in these cases + res = CellComparator.COMPARATOR.compare(bloomKeyKV, lastBloomKeyOnlyKV); } - if (res) { + if (res <= 0) { throw new IOException("Non-increasing Bloom keys: " + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen) + " after " + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset, lastBloomKeyLen)); @@ -1252,7 +1255,10 @@ public class StoreFile { return true; } - byte[] key; + // Used in ROW bloom + byte[] key = null; + // Used in ROW_COL bloom + KeyValue kvKey = null; switch (bloomFilterType) { case ROW: if (col != null) { @@ -1267,7 +1273,7 @@ public class StoreFile { break; case ROWCOL: - key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col, + kvKey = bloomFilter.createBloomKeyValue(row, rowOffset, rowLen, col, colOffset, colLen); break; @@ -1304,9 +1310,7 @@ public class StoreFile { if (bloomFilterType == BloomType.ROW) { keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0); } else { - // TODO : Convert key to Cell so that we could use compare(Cell, Cell) - keyIsAfterLast = (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, key, 0, - key.length)) < 0; + keyIsAfterLast = (CellComparator.COMPARATOR.compare(kvKey, lastBloomKeyOnlyKV)) > 0; } } @@ -1315,19 +1319,17 @@ 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. - byte[] rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen, + KeyValue rowBloomKey = bloomFilter.createBloomKeyValue(row, rowOffset, rowLen, null, 0, 0); // 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 - && (CellComparator.COMPARATOR.compare(lastBloomKeyOnlyKV, rowBloomKey, 0, - rowBloomKey.length)) < 0) { + && (CellComparator.COMPARATOR.compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) { exists = false; } else { exists = - bloomFilter.contains(key, 0, key.length, bloom) || - bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length, - bloom); + bloomFilter.contains(kvKey, bloom) || + bloomFilter.contains(rowBloomKey, bloom); } } else { exists = !keyIsAfterLast diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java index b314bf6..a4de98e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.util; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; /** @@ -45,7 +46,17 @@ public interface BloomFilter extends BloomFilterBase { /** * Check if the specified key is contained in the bloom filter. - * + * Used in ROW_COL blooms where the blooms are serialized as KeyValues + * @param kvKey the keyvalue to check for the existence of + * @param bloom bloom filter data to search. This can be null if auto-loading + * is supported. + * @return true if matched by bloom, false if not + */ + boolean contains(KeyValue kvKey, ByteBuffer bloom); + + /** + * Check if the specified key is contained in the bloom filter. + * Used in ROW bloom where the blooms are just plain byte[] * @param buf data to check for existence of * @param offset offset into the data * @param length length of the data diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java index 0396965..dc98948 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterBase.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.util; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; /** @@ -45,7 +46,7 @@ public interface BloomFilterBase { /** * Create a key for a row-column Bloom filter. */ - byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, + // It is fine to return a KeyValue instead of a Cell because blooms would expect contiguous byte[] + KeyValue createBloomKeyValue(byte[] rowBuf, int rowOffset, int rowLen, byte[] qualBuf, int qualOffset, int qualLen); - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java index 1a63f6b..aecbdf8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java @@ -99,13 +99,6 @@ public final class BloomFilterFactory { throws IllegalArgumentException, IOException { int version = meta.readInt(); switch (version) { - case ByteBloomFilter.VERSION: - // This is only possible in a version 1 HFile. We are ignoring the - // passed comparator because raw byte comparators are always used - // in version 1 Bloom filters. - // TODO:Remove this code - use only CompoundBloomFilter - return new ByteBloomFilter(meta); - case CompoundBloomFilterBase.VERSION: return new CompoundBloomFilter(meta, reader); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java index 1ea81dd..ba0741c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java @@ -26,6 +26,7 @@ import java.nio.ByteBuffer; import java.text.NumberFormat; import java.util.Random; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.io.Writable; @@ -76,8 +77,7 @@ import org.apache.hadoop.io.Writable; * Space/Time Trade-Offs in Hash Coding with Allowable Errors */ @InterfaceAudience.Private -// TODO : Remove this ByteBloomFilter as an instance of BloomFilter -public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { +public class ByteBloomFilter implements BloomFilterWriter { /** Current file format version */ public static final int VERSION = 1; @@ -399,7 +399,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { return contains(buf, 0, buf.length, bloom); } - @Override public boolean contains(byte[] buf, int offset, int length, ByteBuffer theBloom) { if (theBloom == null) { // In a version 1 HFile Bloom filter data is stored in a separate meta @@ -591,11 +590,6 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { return hashCount; } - @Override - public boolean supportsAutoLoading() { - return bloom != null; - } - public static void setFakeLookupMode(boolean enabled) { if (enabled) { randomGeneratorForTest = new Random(283742987L); @@ -610,17 +604,10 @@ public class ByteBloomFilter implements BloomFilter, BloomFilterWriter { * buffer if the column qualifier is empty. */ @Override - public byte[] createBloomKey(byte[] rowBuf, int rowOffset, int rowLen, + public KeyValue createBloomKeyValue(byte[] rowBuf, int rowOffset, int rowLen, byte[] qualBuf, int qualOffset, int qualLen) { - // Optimize the frequent case when only the row is provided. - if (qualLen <= 0 && rowOffset == 0 && rowLen == rowBuf.length) - return rowBuf; - - byte [] result = new byte[rowLen + qualLen]; - System.arraycopy(rowBuf, rowOffset, result, 0, rowLen); - if (qualLen > 0) - System.arraycopy(qualBuf, qualOffset, result, rowLen, qualLen); - return result; + // Ideally should not be called here + return null; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java index cae9d31..e37d819 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilter.java @@ -23,6 +23,7 @@ import java.io.DataInput; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.io.hfile.FixedFileTrailer; @@ -90,10 +91,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuffer bloom) { // 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. - boolean result; int block = index.rootBlockContainingKey(key, keyOffset, - keyLength, comparator); + keyLength); + return checkContains(key, keyOffset, keyLength, block); + } + + private boolean checkContains(byte[] key, int keyOffset, int keyLength, int block) { + boolean result; if (block < 0) { result = false; // This key is not in the file. } else { @@ -126,6 +131,15 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase return result; } + @Override + public boolean contains(KeyValue kvKey, ByteBuffer bloom) { + // 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(kvKey); + byte[] key = kvKey.getKey(); + return checkContains(key, 0, key.length, block); + } + public boolean supportsAutoLoading() { return true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java index d68c78d..4c9a50f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java @@ -77,16 +77,14 @@ public class CompoundBloomFilterBase implements BloomFilterBase { * filters. */ @Override - public byte[] createBloomKey(byte[] row, int roffset, int rlength, - byte[] qualifier, int qoffset, int qlength) { + public KeyValue createBloomKeyValue(byte[] row, int roffset, int rlength, byte[] qualifier, + int qoffset, int qlength) { if (qualifier == null) qualifier = DUMMY; // Make sure this does not specify a timestamp so that the default maximum // (most recent) timestamp is used. - KeyValue kv = KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0, - qualifier, qoffset, qlength); - return kv.getKey(); + return KeyValueUtil.createFirstOnRow(row, roffset, rlength, DUMMY, 0, 0, qualifier, qoffset, + qlength); } - } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 9bd9099..7c4ba4f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -353,10 +353,10 @@ public class TestCompoundBloomFilter { CompoundBloomFilterBase cbfb = new CompoundBloomFilterBase(); byte[] row = "myRow".getBytes(); byte[] qualifier = "myQualifier".getBytes(); - byte[] rowKey = cbfb.createBloomKey(row, 0, row.length, - row, 0, 0); - byte[] rowColKey = cbfb.createBloomKey(row, 0, row.length, - qualifier, 0, qualifier.length); + byte[] rowKey = cbfb.createBloomKeyValue(row, 0, row.length, + row, 0, 0).getKey(); + byte[] rowColKey = cbfb.createBloomKeyValue(row, 0, row.length, + qualifier, 0, qualifier.length).getKey(); KeyValue rowKV = KeyValueUtil.createKeyValueFromKey(rowKey); KeyValue rowColKV = KeyValueUtil.createKeyValueFromKey(rowColKey); assertEquals(rowKV.getTimestamp(), rowColKV.getTimestamp());