Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -423,8 +423,12 @@ private WriterLength getNewWriter(byte[] tableName, byte[] family, Configuration
BloomType bloomType = bloomTypeMap.get(tableAndFamily);
bloomType = bloomType == null ? BloomType.NONE : bloomType;
String bloomParam = bloomParamMap.get(tableAndFamily);
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, bloomParam);
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH && bloomParam != null) {
String[] parts = bloomParam.split(":");
conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, parts[0]);
if (parts.length > 1) {
conf.set(BloomFilterUtil.PREFIX_START_OFFSET_KEY, parts[1]);
}
}
Integer blockSize = blockSizeMap.get(tableAndFamily);
blockSize = blockSize == null ? HConstants.DEFAULT_BLOCKSIZE : blockSize;
Expand Down Expand Up @@ -1018,7 +1022,15 @@ static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, St
BloomType bloomType = familyDescriptor.getBloomFilterType();
String bloomParam = "";
if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {
bloomParam = familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_LENGTH_KEY);
String prefixLength =
familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_LENGTH_KEY);
String startOffset =
familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_START_OFFSET_KEY);
if (startOffset != null && Integer.parseInt(startOffset) > 0) {
bloomParam = prefixLength + ":" + startOffset;
} else {
bloomParam = prefixLength;
}
}
return bloomParam;
};
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ public class StoreFileReader {
private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
private boolean skipResetSeqId = true;
private int prefixLength = -1;
private int prefixStartOffset = 0;
protected Configuration conf;

/**
Expand Down Expand Up @@ -339,19 +340,24 @@ private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {

byte[] row = scan.getStartRow();
byte[] rowPrefix;
if (row.length <= prefixStartOffset) {
return true;
}
if (scan.isGetScan()) {
rowPrefix = Bytes.copy(row, 0, Math.min(prefixLength, row.length));
int len = Math.min(prefixLength, row.length - prefixStartOffset);
rowPrefix = Bytes.copy(row, prefixStartOffset, len);
} else {
// For non-get scans
// Find out the common prefix of startRow and stopRow.
// We find common prefix from the beginning of the row, not from prefixStartOffset,
// because if the bytes before the offset differ, rows in between could have any
// prefix after offset.
int commonLength = Bytes.findCommonPrefix(scan.getStartRow(), scan.getStopRow(),
scan.getStartRow().length, scan.getStopRow().length, 0, 0);
// startRow and stopRow don't have the common prefix.
// Or the common prefix length is less than prefixLength
if (commonLength <= 0 || commonLength < prefixLength) {
if (commonLength <= 0 || commonLength < prefixStartOffset + prefixLength) {
return true;
}
rowPrefix = Bytes.copy(row, 0, prefixLength);
rowPrefix = Bytes.copy(row, prefixStartOffset, prefixLength);
}
return checkGeneralBloomFilter(rowPrefix, null, bloomFilter);
}
Expand Down Expand Up @@ -466,6 +472,7 @@ public Map<byte[], byte[]> loadFileInfo() throws IOException {
byte[] p = fi.get(BLOOM_FILTER_PARAM_KEY);
if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) {
prefixLength = Bytes.toInt(p);
prefixStartOffset = p.length >= 8 ? Bytes.toInt(p, 4) : 0;
}

lastBloomKey = fi.get(LAST_BLOOM_KEY);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -543,7 +543,8 @@ private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf
if (LOG.isTraceEnabled()) {
LOG.trace("Bloom filter type for " + path + ": " + this.bloomType + ", param: "
+ (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH
? Bytes.toInt(bloomParam)
? "prefixLength=" + Bytes.toInt(bloomParam, 0) + ", startOffset="
+ (bloomParam.length >= 8 ? Bytes.toInt(bloomParam, 4) : 0)
: Bytes.toStringBinary(bloomParam))
+ ", " + generalBloomFilterWriter.getClass().getSimpleName());
}
Expand All @@ -558,8 +559,10 @@ private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf
new RowColBloomContext(generalBloomFilterWriter, fileContext.getCellComparator());
break;
case ROWPREFIX_FIXED_LENGTH:
int prefixLen = Bytes.toInt(bloomParam, 0);
int startOff = bloomParam.length >= 8 ? Bytes.toInt(bloomParam, 4) : 0;
bloomContext = new RowPrefixFixedLengthBloomContext(generalBloomFilterWriter,
fileContext.getCellComparator(), Bytes.toInt(bloomParam));
fileContext.getCellComparator(), prefixLen, startOff);
break;
default:
throw new IOException(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public final class BloomFilterUtil {
private static Random randomGeneratorForTest;

public static final String PREFIX_LENGTH_KEY = "RowPrefixBloomFilter.prefix_length";
public static final String PREFIX_START_OFFSET_KEY = "RowPrefixBloomFilter.prefix_start_offset";

/** Bit-value lookup array to prevent doing the same work over and over */
public static final byte[] bitvals = { (byte) 0x01, (byte) 0x02, (byte) 0x04, (byte) 0x08,
Expand Down Expand Up @@ -264,17 +265,25 @@ public static byte[] getBloomFilterParam(BloomType bloomFilterType, Configuratio
int prefixLength;
try {
prefixLength = Integer.parseInt(prefixLengthString);
if (prefixLength <= 0 || prefixLength > HConstants.MAX_ROW_LENGTH) {
message +=
"the value of " + PREFIX_LENGTH_KEY + " must >=0 and < " + HConstants.MAX_ROW_LENGTH;
if (prefixLength <= 0) {
message += "the value of " + PREFIX_LENGTH_KEY + " must be > 0";
throw new IllegalArgumentException(message);
}
} catch (NumberFormatException nfe) {
message = "Number format exception when parsing " + PREFIX_LENGTH_KEY + " for BloomType "
+ bloomFilterType.toString() + ":" + prefixLengthString;
throw new IllegalArgumentException(message, nfe);
}
bloomParam = Bytes.toBytes(prefixLength);
int startOffset = conf.getInt(PREFIX_START_OFFSET_KEY, 0);
if (startOffset < 0) {
throw new IllegalArgumentException(
message + "the value of " + PREFIX_START_OFFSET_KEY + " must be >= 0");
}
if ((long) startOffset + prefixLength > HConstants.MAX_ROW_LENGTH) {
throw new IllegalArgumentException(message + "the sum of " + PREFIX_START_OFFSET_KEY
+ " and " + PREFIX_LENGTH_KEY + " must be <= " + HConstants.MAX_ROW_LENGTH);
}
bloomParam = Bytes.add(Bytes.toBytes(prefixLength), Bytes.toBytes(startOffset));
}
return bloomParam;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,15 +33,23 @@
@InterfaceAudience.Private
public class RowPrefixFixedLengthBloomContext extends RowBloomContext {
private final int prefixLength;
private final int startOffset;

public RowPrefixFixedLengthBloomContext(BloomFilterWriter bloomFilterWriter,
CellComparator comparator, int prefixLength) {
CellComparator comparator, int prefixLength, int startOffset) {
super(bloomFilterWriter, comparator);
this.prefixLength = prefixLength;
this.startOffset = startOffset;
}

@Override
public void writeBloom(ExtendedCell cell) throws IOException {
// Skip rows shorter than startOffset. They have no bytes at the prefix window, and the read
// path bypasses the bloom entirely when the query row is this short (returns true to scan),
// so any entry we would write here is unreachable and only inflates the false positive rate.
if (cell.getRowLength() <= startOffset) {
return;
}
super.writeBloom(getRowPrefixCell(cell));
}

Expand All @@ -51,7 +59,8 @@ public void writeBloom(ExtendedCell cell) throws IOException {
*/
private ExtendedCell getRowPrefixCell(ExtendedCell cell) {
byte[] row = CellUtil.copyRow(cell);
int length = Math.min(prefixLength, row.length - startOffset);
return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setRow(row, 0, Math.min(prefixLength, row.length)).setType(Cell.Type.Put).build();
.setRow(row, startOffset, length).setType(Cell.Type.Put).build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,178 @@ public void testRowPrefixBloomFilterWithGet() throws Exception {
fs.delete(f, true);
}

@Test
public void testRowPrefixBloomFilterWithStartOffset() throws Exception {
FileSystem fs = FileSystem.getLocal(conf);
int startOffset = 2;
int offsetPrefixLength = 4;
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, offsetPrefixLength);
conf.setInt(BloomFilterUtil.PREFIX_START_OFFSET_KEY, startOffset);
try {
if (!fs.exists(testDir)) {
fs.mkdirs(testDir);
}
Path f = StoreFileWriter.getUniqueFile(fs, testDir);

// Write rows with 2-byte salt + 4-byte meaningful prefix + suffix.
// Use two different salts ("AA" and "AB") with the same meaningful prefixes
// to verify the bloom filter correctly skips the salt bytes.
int expKeys = 50;
HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL)
.withChecksumType(CKTYPE).withBytesPerCheckSum(CKBYTES).build();
StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(f)
.withBloomType(bt).withMaxKeyCount(expKeys).withFileContext(meta).build();
long now = EnvironmentEdgeManager.currentTime();
try {
// Write rows with two salts ("AA" and "AB") sharing the same meaningful
// prefixes. Assign non-overlapping prefix ranges per salt so bloom keys
// (the meaningful prefix after offset) stay sorted across the file.
for (int i = 0; i < 50; i += 2) {
String prefix = String.format("%04d", i);
for (int j = 0; j < 3; j++) {
String row = "AA" + prefix + String.format("_%03d", j);
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
writer.append(kv);
}
}
for (int i = 50; i < 100; i += 2) {
String prefix = String.format("%04d", i);
for (int j = 0; j < 3; j++) {
String row = "AB" + prefix + String.format("_%03d", j);
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
writer.append(kv);
}
}
} finally {
writer.close();
}

ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, f).build();
StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, f, true);
storeFileInfo.initHFileInfo(context);
StoreFileReader reader = storeFileInfo.createReader(context, cacheConf);
storeFileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader());
reader.loadFileInfo();
reader.loadBloomfilter();

StoreFileScanner scanner = getStoreFileScanner(reader);
HStore store = mock(HStore.class);
when(store.getColumnFamilyDescriptor())
.thenReturn(ColumnFamilyDescriptorBuilder.of("family"));

// "AA" salt, even prefix written under "AA" range - in bloom
Scan scan = new Scan(new Get(Bytes.toBytes("AA0002_000")));
assertTrue(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// "AA" salt, odd prefix - NOT in bloom
scan = new Scan(new Get(Bytes.toBytes("AA0003_000")));
assertFalse(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// "AB" salt, even prefix written under "AB" range - in bloom
scan = new Scan(new Get(Bytes.toBytes("AB0050_000")));
assertTrue(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// "AB" salt, odd prefix - NOT in bloom
scan = new Scan(new Get(Bytes.toBytes("AB0051_000")));
assertFalse(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// Prefix "0002" was written under salt "AA". Query with salt "AB" but same
// meaningful prefix. Since bloom only hashes the prefix after offset,
// this should still match the bloom.
scan = new Scan(new Get(Bytes.toBytes("AB0002_000")));
assertTrue(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

reader.close(true);
fs.delete(f, true);
} finally {
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, prefixLength);
conf.unset(BloomFilterUtil.PREFIX_START_OFFSET_KEY);
}
}

@Test
public void testRowPrefixBloomFilterWithStartOffsetScan() throws Exception {
FileSystem fs = FileSystem.getLocal(conf);
int startOffset = 2;
int offsetPrefixLength = 4;
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, offsetPrefixLength);
conf.setInt(BloomFilterUtil.PREFIX_START_OFFSET_KEY, startOffset);
try {
if (!fs.exists(testDir)) {
fs.mkdirs(testDir);
}
Path f = StoreFileWriter.getUniqueFile(fs, testDir);

int expKeys = 50;
HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL)
.withChecksumType(CKTYPE).withBytesPerCheckSum(CKBYTES).build();
StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(f)
.withBloomType(bt).withMaxKeyCount(expKeys).withFileContext(meta).build();
long now = EnvironmentEdgeManager.currentTime();
try {
// Write rows with two salts ("AA" and "AB") sharing the same meaningful
// prefixes. Assign non-overlapping prefix ranges per salt so bloom keys
// (the meaningful prefix after offset) stay sorted across the file.
for (int i = 0; i < 50; i += 2) {
String prefix = String.format("%04d", i);
for (int j = 0; j < 3; j++) {
String row = "AA" + prefix + String.format("_%03d", j);
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
writer.append(kv);
}
}
for (int i = 50; i < 100; i += 2) {
String prefix = String.format("%04d", i);
for (int j = 0; j < 3; j++) {
String row = "AB" + prefix + String.format("_%03d", j);
KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
Bytes.toBytes("col"), now, Bytes.toBytes("value"));
writer.append(kv);
}
}
} finally {
writer.close();
}

ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, f).build();
StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, f, true);
storeFileInfo.initHFileInfo(context);
StoreFileReader reader = storeFileInfo.createReader(context, cacheConf);
storeFileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader());
reader.loadFileInfo();
reader.loadBloomfilter();

StoreFileScanner scanner = getStoreFileScanner(reader);
HStore store = mock(HStore.class);
when(store.getColumnFamilyDescriptor())
.thenReturn(ColumnFamilyDescriptorBuilder.of("family"));

// Scan with even prefix under "AB" range (in bloom)
Scan scan = new Scan().withStartRow(Bytes.toBytes("AB0050_000"))
.withStopRow(Bytes.toBytes("AB0050_002"));
assertTrue(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// Scan with odd prefix under "AB" range (NOT in bloom)
scan = new Scan().withStartRow(Bytes.toBytes("AB0051_000"))
.withStopRow(Bytes.toBytes("AB0051_002"));
assertFalse(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

// Scan where common prefix is less than offset + prefixLength (should pass through)
scan = new Scan().withStartRow(Bytes.toBytes("AB0050_000"))
.withStopRow(Bytes.toBytes("AB0052_000"));
assertTrue(scanner.shouldUseScanner(scan, store, Long.MIN_VALUE));

reader.close(true);
fs.delete(f, true);
} finally {
conf.setInt(BloomFilterUtil.PREFIX_LENGTH_KEY, prefixLength);
conf.unset(BloomFilterUtil.PREFIX_START_OFFSET_KEY);
}
}

@Test
public void testRowPrefixBloomFilterWithScan() throws Exception {
FileSystem fs = FileSystem.getLocal(conf);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -236,14 +236,33 @@ Bloom filters need to be rebuilt upon deletion, so may not be appropriate in env

#### Enabling Bloom Filters

Bloom filters are enabled on a Column Family. You can do this by using the setBloomFilterType method of HColumnDescriptor or using the HBase API. Valid values are `NONE`, `ROW` (default), or `ROWCOL`. See [When To Use Bloom Filters](/docs/performance#when-to-use-bloom-filters) for more information on `ROW` versus `ROWCOL`. See also the API documentation for [ColumnFamilyDescriptorBuilder](https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.html).
Bloom filters are enabled on a Column Family. You can do this by using the setBloomFilterType method of HColumnDescriptor or using the HBase API. Valid values are `NONE`, `ROW` (default), `ROWCOL`, or `ROWPREFIX_FIXED_LENGTH`. See [When To Use Bloom Filters](/docs/performance#when-to-use-bloom-filters) for more information on `ROW` versus `ROWCOL`. See also the API documentation for [ColumnFamilyDescriptorBuilder](https://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/client/ColumnFamilyDescriptorBuilder.html).

The following example creates a table and enables a ROWCOL Bloom filter on the `colfam1` column family.

```ruby
hbase> create 'mytable',{NAME => 'colfam1', BLOOMFILTER => 'ROWCOL'}
```

##### ROWPREFIX_FIXED_LENGTH Bloom Filter

`ROWPREFIX_FIXED_LENGTH` is useful when row keys share a common prefix structure. The bloom filter is built on a fixed-length prefix of the row key rather than the entire row. This reduces the number of entries in the bloom filter when many rows share the same prefix.

The following configuration properties are available:

| Parameter | Default | Description |
| ------------------------------------------ | ------- | -------------------------------------------------------------------------------------------------------- |
| RowPrefixBloomFilter.prefix_length | (none) | Required. The length of the row key prefix to use for the bloom filter. |
| RowPrefixBloomFilter.prefix_start_offset | 0 | The starting byte offset of the prefix within the row key. Useful for salted or hash-prefixed row keys. |

For example, if your row keys have a 2-byte salt followed by a 4-byte meaningful prefix (e.g., `<2-byte-salt><4-byte-prefix><rest>`), you can set `prefix_start_offset=2` and `prefix_length=4` to build the bloom filter on the meaningful prefix only.

```ruby
hbase> create 'mytable',{NAME => 'colfam1', BLOOMFILTER => 'ROWPREFIX_FIXED_LENGTH',
CONFIGURATION => {'RowPrefixBloomFilter.prefix_length' => '4',
'RowPrefixBloomFilter.prefix_start_offset' => '2'}}
```

#### Configuring Server-Wide Behavior of Bloom Filters

You can configure the following settings in the _hbase-site.xml_.
Expand Down