defaultValueMap = ColumnFamilyDescriptorBuilder.getDefaultValues();
- assertEquals(defaultValueMap.size(), 12);
+ assertEquals(defaultValueMap.size(), 13);
assertEquals(defaultValueMap.get(ColumnFamilyDescriptorBuilder.BLOOMFILTER),
BloomType.ROW.toString());
+ assertEquals(defaultValueMap.get(ColumnFamilyDescriptorBuilder.BLOOMFILTER_IMPL),
+ BloomFilterImpl.BLOOM.toString());
assertEquals(defaultValueMap.get(ColumnFamilyDescriptorBuilder.REPLICATION_SCOPE), "0");
assertEquals(defaultValueMap.get(ColumnFamilyDescriptorBuilder.MAX_VERSIONS), "1");
assertEquals(defaultValueMap.get(ColumnFamilyDescriptorBuilder.MIN_VERSIONS), "0");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilter.java
new file mode 100644
index 000000000000..a5b175e07a0b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilter.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import java.io.DataInput;
+import java.io.IOException;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.BloomFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ribbon.InterleavedRibbonSolution;
+import org.apache.hadoop.hbase.util.ribbon.RibbonFilterUtil;
+import org.apache.hadoop.hbase.util.ribbon.RibbonHasher;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Reader for compound Ribbon filters stored in HFiles.
+ *
+ * This class provides on-demand loading of Ribbon filter chunks, similar to
+ * {@link CompoundBloomFilter}. Each chunk is loaded from the HFile block cache when needed for a
+ * query.
+ *
+ */
+@InterfaceAudience.Private
+public class CompoundRibbonFilter extends CompoundRibbonFilterBase implements BloomFilter {
+
+ /** HFile reader for loading chunks on demand */
+ private final HFile.Reader reader;
+
+ /** Metrics collector */
+ private final BloomFilterMetrics metrics;
+
+ /** Block index for locating chunks */
+ private final HFileBlockIndex.BlockIndexReader index;
+
+ /** Per-chunk metadata for queries */
+ private final int[] chunkNumSlots;
+
+ /** Per-chunk upperNumColumns for ICML mode */
+ private final int[] chunkUpperNumColumns;
+
+ /** Per-chunk upperStartBlock for ICML mode */
+ private final int[] chunkUpperStartBlock;
+
+ /**
+ * Deserializes a CompoundRibbonFilter from HFile metadata.
+ * @param meta DataInput positioned at the start of Ribbon filter metadata (after version)
+ * @param reader HFile reader for loading chunks
+ * @param metrics Metrics collector (may be null)
+ * @throws IOException If an I/O error occurs
+ */
+ public CompoundRibbonFilter(DataInput meta, HFile.Reader reader, BloomFilterMetrics metrics)
+ throws IOException {
+ this.reader = reader;
+ this.metrics = metrics;
+
+ // Read metadata (must match CompoundRibbonFilterWriter.MetaWriter.write())
+ totalByteSize = meta.readLong();
+ bandwidth = meta.readInt();
+ hashType = meta.readInt();
+ overheadRatio = meta.readDouble();
+ totalKeyCount = meta.readLong();
+ totalNumSlots = meta.readLong();
+ numChunks = meta.readInt();
+
+ // Read comparator class name
+ byte[] comparatorClassName = Bytes.readByteArray(meta);
+ if (comparatorClassName.length != 0) {
+ comparator = FixedFileTrailer.createComparator(Bytes.toString(comparatorClassName));
+ }
+
+ // Read per-chunk numSlots array
+ chunkNumSlots = new int[numChunks];
+ for (int i = 0; i < numChunks; i++) {
+ chunkNumSlots[i] = meta.readInt();
+ }
+
+ // Read ICML per-chunk metadata
+ chunkUpperNumColumns = new int[numChunks];
+ chunkUpperStartBlock = new int[numChunks];
+ for (int i = 0; i < numChunks; i++) {
+ chunkUpperNumColumns[i] = meta.readInt();
+ chunkUpperStartBlock[i] = meta.readInt();
+ }
+
+ // Initialize block index reader
+ if (comparator == null) {
+ index = new HFileBlockIndex.ByteArrayKeyBlockIndexReader(1);
+ } else {
+ index = new HFileBlockIndex.CellBasedKeyBlockIndexReader(comparator, 1);
+ }
+ index.readRootIndex(meta, numChunks);
+ }
+
+ @Override
+ public boolean contains(Cell keyCell, ByteBuff bloom, BloomType type) {
+ boolean result = containsInternal(keyCell, type);
+ if (metrics != null) {
+ metrics.incrementRequests(result);
+ }
+ return result;
+ }
+
+ private boolean containsInternal(Cell keyCell, BloomType type) {
+ byte[] key = RibbonFilterUtil.extractKeyFromCell(keyCell, type);
+
+ // Find block using appropriate index type
+ int block;
+ if (comparator != null) {
+ block = index.rootBlockContainingKey(keyCell);
+ } else {
+ block = index.rootBlockContainingKey(key, 0, key.length);
+ }
+
+ return containsInternal(block, key, 0, key.length);
+ }
+
+ @Override
+ public boolean contains(byte[] buf, int offset, int length, ByteBuff bloom) {
+ boolean result = containsInternal(buf, offset, length);
+ if (metrics != null) {
+ metrics.incrementRequests(result);
+ }
+ return result;
+ }
+
+ private boolean containsInternal(byte[] key, int keyOffset, int keyLength) {
+ int block = index.rootBlockContainingKey(key, keyOffset, keyLength);
+ return containsInternal(block, key, keyOffset, keyLength);
+ }
+
+ private boolean containsInternal(int block, byte[] key, int keyOffset, int keyLength) {
+ if (block < 0) {
+ return false;
+ }
+
+ HFileBlock ribbonBlock = loadRibbonBlock(block);
+ try {
+ ByteBuff buf = ribbonBlock.getBufferReadOnly();
+ int headerSize = ribbonBlock.headerSize();
+ int numSlots = getChunkNumSlots(block, ribbonBlock);
+
+ RibbonHasher hasher = new RibbonHasher(numSlots, bandwidth, hashType);
+
+ RibbonHasher.RibbonHashResult hashResult = hasher.hash(key, keyOffset, keyLength);
+
+ return InterleavedRibbonSolution.contains(hashResult.start(), hashResult.coeffRow(),
+ hashResult.resultRow(), buf, headerSize, numSlots, chunkUpperNumColumns[block],
+ chunkUpperStartBlock[block]);
+ } finally {
+ ribbonBlock.release();
+ }
+ }
+
+ /**
+ * Loads a Ribbon filter block from the HFile.
+ * @param blockIndex Index of the block to load
+ * @return The loaded HFile block containing Ribbon filter data
+ */
+ private HFileBlock loadRibbonBlock(int blockIndex) {
+ try {
+ return reader.readBlock(index.getRootBlockOffset(blockIndex),
+ index.getRootBlockDataSize(blockIndex), true, // cacheBlock
+ true, // pread
+ false, // isCompaction
+ true, // updateCacheMetrics
+ BlockType.BLOOM_CHUNK, null // expectedDataBlockEncoding
+ );
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Failed to load Ribbon block", e);
+ }
+ }
+
+ /**
+ * Gets the number of slots for a chunk.
+ *
+ * With byte-based format (1 byte per slot), the relationship is: dataSize = numSlots bytes.
+ *
+ * Uses precomputed chunkNumSlots array from metadata if available, otherwise uses block size
+ * directly.
+ * @param blockIndex Index of the block
+ * @param block The HFile block (used as fallback if metadata unavailable)
+ * @return Number of slots in this chunk
+ */
+ private int getChunkNumSlots(int blockIndex, HFileBlock block) {
+ // Use precomputed value if available
+ if (blockIndex < chunkNumSlots.length && chunkNumSlots[blockIndex] > 0) {
+ return chunkNumSlots[blockIndex];
+ }
+
+ // Fallback: dataSize equals numSlots (1 byte per slot)
+ return block.getUncompressedSizeWithoutHeader();
+ }
+
+ @Override
+ public boolean supportsAutoLoading() {
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("CompoundRibbonFilter");
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Keys: ").append(totalKeyCount);
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Slots: ").append(totalNumSlots);
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Chunks: ").append(numChunks);
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Bandwidth: ").append(bandwidth);
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Byte size: ").append(totalByteSize);
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Overhead: ").append(String.format("%.2f%%", overheadRatio * 100));
+ if (comparator != null) {
+ sb.append(RibbonFilterUtil.STATS_RECORD_SEP);
+ sb.append("Comparator: ").append(comparator.getClass().getSimpleName());
+ }
+ return sb.toString();
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterBase.java
new file mode 100644
index 000000000000..9771aa43af9b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterBase.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.util.BloomFilterBase;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Base class for Compound Ribbon Filter implementations. This class provides common fields and
+ * methods shared between the reader ({@link CompoundRibbonFilter}) and writer
+ * ({@link CompoundRibbonFilterWriter}).
+ */
+@InterfaceAudience.Private
+public class CompoundRibbonFilterBase implements BloomFilterBase {
+
+ /**
+ * The Ribbon filter version. This is different from CompoundBloomFilterBase.VERSION to
+ * distinguish Ribbon filters from Bloom filters when reading metadata.
+ */
+ public static final int VERSION = 101;
+
+ /**
+ * At read time, the total number of chunks. At write time, the number of chunks created so far.
+ * The first chunk has an ID of 0, and the current chunk has the ID of numChunks - 1.
+ */
+ protected int numChunks;
+
+ /** The total number of keys in all chunks */
+ protected long totalKeyCount;
+
+ /** The total byte size of all chunks */
+ protected long totalByteSize;
+
+ /** The total number of slots across all chunks */
+ protected long totalNumSlots;
+
+ /** Bandwidth (coefficient width in bits), typically 64 */
+ protected int bandwidth;
+
+ /** Space overhead ratio (e.g., 0.05 for 5%) */
+ protected double overheadRatio;
+
+ /** Hash function type to use, as defined in {@link org.apache.hadoop.hbase.util.Hash} */
+ protected int hashType;
+
+ /** Comparator used to compare filter keys (for ROWCOL type) */
+ protected CellComparator comparator;
+
+ @Override
+ public long getMaxKeys() {
+ // For Ribbon filters, maxKeys equals keyCount since we build to fit exactly
+ return totalKeyCount;
+ }
+
+ @Override
+ public long getKeyCount() {
+ return totalKeyCount;
+ }
+
+ @Override
+ public long getByteSize() {
+ return totalByteSize;
+ }
+
+ /**
+ * Returns the bandwidth (coefficient width in bits).
+ */
+ public int getBandwidth() {
+ return bandwidth;
+ }
+
+ /**
+ * Returns the space overhead ratio.
+ */
+ public double getOverheadRatio() {
+ return overheadRatio;
+ }
+
+ /**
+ * Returns the number of chunks.
+ */
+ public int getNumChunks() {
+ return numChunks;
+ }
+
+ /**
+ * Returns the hash type.
+ */
+ public int getHashType() {
+ return hashType;
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterWriter.java
new file mode 100644
index 000000000000..4c4d5942e604
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundRibbonFilterWriter.java
@@ -0,0 +1,370 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.io.hfile;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Queue;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.BloomFilterWriter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ribbon.InterleavedRibbonSolution;
+import org.apache.hadoop.hbase.util.ribbon.RibbonFilterChunk;
+import org.apache.hadoop.hbase.util.ribbon.RibbonFilterUtil;
+import org.apache.hadoop.io.Writable;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Writer for compound Ribbon filters in HFiles.
+ *
+ * This class manages the lifecycle of Ribbon filter chunks similar to
+ * {@link CompoundBloomFilterWriter}. Keys are buffered in each chunk and processed during
+ * finalization when the chunk is full.
+ *
+ */
+@InterfaceAudience.Private
+public class CompoundRibbonFilterWriter extends CompoundRibbonFilterBase
+ implements BloomFilterWriter, InlineBlockWriter {
+
+ private static final Logger LOG = LoggerFactory.getLogger(CompoundRibbonFilterWriter.class);
+
+ /** Target block size in bytes (same as Bloom filter block size) */
+ private final int blockSize;
+
+ /** Maximum number of keys per chunk (computed from blockSize) */
+ private final int maxKeysPerChunk;
+
+ /** Desired false positive rate */
+ private final double desiredFpRate;
+
+ /** Pre-calculated optimal overhead (calculated once in constructor) */
+ private final double overhead;
+
+ /** The current chunk being written to */
+ private RibbonFilterChunk chunk;
+
+ /** Previous chunk, for creating another similar chunk */
+ private RibbonFilterChunk prevChunk;
+
+ /** The first key in the current chunk */
+ private byte[] firstKeyInChunk;
+
+ /** The previous cell that was processed */
+ private ExtendedCell prevCell;
+
+ /** Whether to cache-on-write compound Ribbon filter chunks */
+ private final boolean cacheOnWrite;
+
+ /** The bloom type for key extraction */
+ private final BloomType bloomType;
+
+ /**
+ * A Ribbon filter chunk ready for writing.
+ *
+ * This class holds the constructed Ribbon filter chunk along with its metadata (first key, ICML
+ * parameters) until it can be written to the HFile.
+ */
+ private static final class ReadyChunk {
+ int chunkId;
+ byte[] firstKey;
+ RibbonFilterChunk chunk;
+ // ICML metadata
+ int upperNumColumns;
+ int upperStartBlock;
+ }
+
+ /** Queue of chunks ready to be written */
+ private final Queue readyChunks = new ArrayDeque<>();
+
+ /** Block index writer for chunk offsets */
+ private final HFileBlockIndex.BlockIndexWriter ribbonBlockIndexWriter =
+ new HFileBlockIndex.BlockIndexWriter();
+
+ /** Per-chunk numSlots for accurate metadata storage */
+ private final List chunkNumSlotsList = new ArrayList<>();
+
+ /** Per-chunk upperNumColumns for ICML */
+ private final List chunkUpperNumColumnsList = new ArrayList<>();
+
+ /** Per-chunk upperStartBlock for ICML */
+ private final List chunkUpperStartBlockList = new ArrayList<>();
+
+ /**
+ * Creates a new CompoundRibbonFilterWriter.
+ *
+ * Always uses ICML (Interleaved Column-Major Layout) for space-optimal storage. Overhead ratio is
+ * calculated automatically based on the FP rate.
+ * @param blockSize Target block size in bytes (same as Bloom filter block size)
+ * @param bandwidth Coefficient width in bits (typically 64)
+ * @param hashType Hash function type
+ * @param cacheOnWrite Whether to cache chunks on write
+ * @param comparator Cell comparator (for ROWCOL type)
+ * @param bloomType The bloom/ribbon type
+ * @param desiredFpRate Desired false positive rate (e.g., 0.01 for 1%)
+ */
+ public CompoundRibbonFilterWriter(int blockSize, int bandwidth, int hashType,
+ boolean cacheOnWrite, CellComparator comparator, BloomType bloomType, double desiredFpRate) {
+ this.blockSize = blockSize;
+ this.bandwidth = bandwidth;
+ this.hashType = hashType;
+ this.desiredFpRate = desiredFpRate;
+ this.cacheOnWrite = cacheOnWrite;
+ this.comparator = comparator;
+ this.bloomType = bloomType;
+
+ // Pre-calculate optimal overhead based on FP rate
+ this.overhead = RibbonFilterUtil.computeOptimalOverheadForFpRate(desiredFpRate, bandwidth);
+
+ // Compute maxKeysPerChunk from blockSize
+ this.maxKeysPerChunk = computeMaxKeysFromBlockSize();
+ }
+
+ /**
+ * Computes the maximum number of keys per chunk based on blockSize.
+ *
+ * ICML uses variable bits per slot: maxKeys = blockSize * 8 / ((1 + overhead) * bitsPerKey)
+ */
+ private int computeMaxKeysFromBlockSize() {
+ double effectiveOverhead = overhead > 0 ? overhead : RibbonFilterUtil.MIN_OVERHEAD_RATIO;
+
+ // ICML mode: bits per key = fingerprintBits
+ int bitsPerKey = RibbonFilterUtil.computeFingerprintBits(desiredFpRate);
+ // maxKeys = blockSize * 8 / ((1 + overhead) * bitsPerKey)
+ return (int) (blockSize * 8.0 / ((1.0 + effectiveOverhead) * bitsPerKey));
+ }
+
+ @Override
+ public boolean shouldWriteBlock(boolean closing) {
+ enqueueReadyChunk(closing);
+ return !readyChunks.isEmpty();
+ }
+
+ /**
+ * Enqueue the current chunk if it is ready to be written out.
+ * @param closing true if we are closing the file, so we do not expect new keys to show up
+ */
+ private void enqueueReadyChunk(boolean closing) {
+ if (chunk == null || (chunk.getKeyCount() < maxKeysPerChunk && !closing)) {
+ return;
+ }
+
+ if (firstKeyInChunk == null) {
+ throw new NullPointerException("Trying to enqueue a chunk, but first key is null: closing="
+ + closing + ", keyCount=" + chunk.getKeyCount() + ", maxKeys=" + maxKeysPerChunk);
+ }
+
+ // Finalize the chunk (back-substitution)
+ chunk.finalizeRibbon();
+
+ // Create ready chunk
+ ReadyChunk readyChunk = new ReadyChunk();
+ readyChunk.chunkId = numChunks - 1;
+ readyChunk.firstKey = firstKeyInChunk;
+ readyChunk.chunk = chunk;
+
+ // Store ICML metadata
+ InterleavedRibbonSolution sol = chunk.getInterleavedSolution();
+ if (sol != null) {
+ readyChunk.upperNumColumns = sol.getUpperNumColumns();
+ readyChunk.upperStartBlock = sol.getUpperStartBlock();
+ }
+
+ readyChunks.add(readyChunk);
+
+ // Update totals
+ totalByteSize += chunk.getByteSize();
+ totalNumSlots += chunk.getNumSlots();
+ chunkNumSlotsList.add(chunk.getNumSlots());
+ chunkUpperNumColumnsList.add(readyChunk.upperNumColumns);
+ chunkUpperStartBlockList.add(readyChunk.upperStartBlock);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Finalized Ribbon chunk #{} with {} keys, {} slots, {} bytes", readyChunk.chunkId,
+ chunk.getKeyCount(), chunk.getNumSlots(), chunk.getByteSize());
+ }
+
+ // Reset for next chunk
+ firstKeyInChunk = null;
+ prevChunk = chunk;
+ chunk = null;
+ }
+
+ @Override
+ public void append(ExtendedCell cell) throws IOException {
+ Objects.requireNonNull(cell);
+
+ enqueueReadyChunk(false);
+
+ if (chunk == null) {
+ if (firstKeyInChunk != null) {
+ throw new IllegalStateException(
+ "First key in chunk already set: " + Bytes.toStringBinary(firstKeyInChunk));
+ }
+ // This will be done only once per chunk
+ firstKeyInChunk = RibbonFilterUtil.extractKeyFromCell(cell, bloomType);
+ allocateNewChunk();
+ }
+
+ chunk.add(cell);
+ this.prevCell = cell;
+ ++totalKeyCount;
+ }
+
+ /**
+ * Allocates a new Ribbon filter chunk with pre-allocated banding matrix.
+ */
+ private void allocateNewChunk() {
+ if (prevChunk == null) {
+ // First chunk
+ chunk = new RibbonFilterChunk(bandwidth, hashType, bloomType, desiredFpRate, overhead);
+ } else {
+ // Use the same parameters as the last chunk
+ chunk = prevChunk.createAnother();
+ }
+
+ // Pre-allocate the banding matrix
+ chunk.allocRibbon(maxKeysPerChunk);
+ ++numChunks;
+ }
+
+ @Override
+ public void writeInlineBlock(DataOutput out) throws IOException {
+ ReadyChunk readyChunk = readyChunks.peek();
+ if (readyChunk == null) {
+ throw new IOException("No ready chunk to write");
+ }
+
+ // Write the Ribbon filter data
+ readyChunk.chunk.writeRibbon(out);
+ }
+
+ @Override
+ public void blockWritten(long offset, int onDiskSize, int uncompressedSize) {
+ ReadyChunk readyChunk = readyChunks.remove();
+ ribbonBlockIndexWriter.addEntry(readyChunk.firstKey, offset, onDiskSize);
+ }
+
+ @Override
+ public BlockType getInlineBlockType() {
+ return BlockType.BLOOM_CHUNK; // Reuse BLOOM_CHUNK type for compatibility
+ }
+
+ @Override
+ public boolean getCacheOnWrite() {
+ return cacheOnWrite;
+ }
+
+ @Override
+ public void beforeShipped() throws IOException {
+ if (this.prevCell != null) {
+ this.prevCell = KeyValueUtil.toNewKeyCell(this.prevCell);
+ }
+ }
+
+ @Override
+ public Cell getPrevCell() {
+ return this.prevCell;
+ }
+
+ @Override
+ public void compactBloom() {
+ // No-op for Ribbon filters.
+ // Unlike Bloom filters which can be folded post-construction,
+ // Ribbon filters use lazy allocation to achieve optimal sizing:
+ // the banding matrix is sized based on actual keyCount during
+ // finalizeRibbon(), not the pre-estimated maxKeys.
+ }
+
+ @Override
+ public Writable getMetaWriter() {
+ return new MetaWriter();
+ }
+
+ @Override
+ public Writable getDataWriter() {
+ return null;
+ }
+
+ /**
+ * Metadata writer for Ribbon filter.
+ *
+ * Writes all metadata required to reconstruct the CompoundRibbonFilter at read time, including
+ * filter parameters, per-chunk ICML metadata, and the block index.
+ */
+ private final class MetaWriter implements Writable {
+ /**
+ * Not supported - this is a write-only implementation.
+ * @throws IOException Always thrown
+ */
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ throw new IOException("Cannot read with MetaWriter");
+ }
+
+ /**
+ * Writes Ribbon filter metadata to the output stream.
+ * @param out The output stream to write to
+ * @throws IOException If an I/O error occurs
+ */
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(VERSION);
+
+ // Ribbon-specific metadata
+ out.writeLong(totalByteSize);
+ out.writeInt(bandwidth);
+ out.writeInt(hashType);
+ out.writeDouble(overhead);
+ out.writeLong(totalKeyCount);
+ out.writeLong(totalNumSlots);
+ out.writeInt(numChunks);
+
+ // Comparator class name (for ROWCOL type)
+ if (comparator != null) {
+ Bytes.writeByteArray(out, Bytes.toBytes(comparator.getClass().getName()));
+ } else {
+ Bytes.writeByteArray(out, null);
+ }
+
+ // Write per-chunk numSlots array for accurate slot counts
+ for (int i = 0; i < numChunks; i++) {
+ out.writeInt(chunkNumSlotsList.get(i));
+ }
+
+ // Write ICML per-chunk metadata
+ for (int i = 0; i < numChunks; i++) {
+ out.writeInt(chunkUpperNumColumnsList.get(i));
+ out.writeInt(chunkUpperStartBlockList.get(i));
+ }
+
+ // Write block index
+ ribbonBlockIndexWriter.writeSingleLevelIndex(out, "Ribbon filter");
+ }
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index fde89d122e28..86c86fcd7a44 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -95,6 +95,7 @@
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -338,7 +339,9 @@ private StoreContext initializeStoreContext(ColumnFamilyDescriptor family) throw
return new StoreContext.Builder().withBlockSize(family.getBlocksize())
.withEncryptionContext(SecurityUtil.createEncryptionContext(conf, region.getTableDescriptor(),
family, region.getManagedKeyDataCache(), region.getSystemKeyCache()))
- .withBloomType(family.getBloomFilterType()).withCacheConfig(createCacheConf(family))
+ .withBloomType(family.getBloomFilterType())
+ .withBloomFilterImpl(BloomFilterFactory.getBloomFilterImpl(conf, family))
+ .withCacheConfig(createCacheConf(family))
.withCellComparator(region.getTableDescriptor().isMetaTable() || conf
.getBoolean(HRegion.USE_META_CELL_COMPARATOR, HRegion.DEFAULT_USE_META_CELL_COMPARATOR)
? MetaCellComparator.META_COMPARATOR
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
index a90ec97dc3fa..ac77d73ccfff 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java
@@ -96,6 +96,9 @@ public class HStoreFile implements StoreFile {
/** Bloom filter Type in FileInfo */
public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
+ /** Bloom filter implementation in FileInfo */
+ public static final byte[] BLOOM_FILTER_IMPL_KEY = Bytes.toBytes("BLOOM_FILTER_IMPL");
+
/** Bloom filter param in FileInfo */
public static final byte[] BLOOM_FILTER_PARAM_KEY = Bytes.toBytes("BLOOM_FILTER_PARAM");
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
index 7bb800a1d39c..6ae4fd531789 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreContext.java
@@ -46,6 +46,7 @@ public final class StoreContext implements HeapSize {
private final HRegionFileSystem regionFileSystem;
private final CellComparator comparator;
private final BloomType bloomFilterType;
+ private final BloomFilterImpl bloomFilterImpl;
private final Supplier> compactedFilesSupplier;
private final Supplier favoredNodesSupplier;
private final ColumnFamilyDescriptor family;
@@ -59,6 +60,7 @@ private StoreContext(Builder builder) {
this.regionFileSystem = builder.regionFileSystem;
this.comparator = builder.comparator;
this.bloomFilterType = builder.bloomFilterType;
+ this.bloomFilterImpl = builder.bloomFilterImpl;
this.compactedFilesSupplier = builder.compactedFilesSupplier;
this.favoredNodesSupplier = builder.favoredNodesSupplier;
this.family = builder.family;
@@ -90,6 +92,10 @@ public BloomType getBloomFilterType() {
return bloomFilterType;
}
+ public BloomFilterImpl getBloomFilterImpl() {
+ return bloomFilterImpl;
+ }
+
public Supplier> getCompactedFilesSupplier() {
return compactedFilesSupplier;
}
@@ -146,6 +152,7 @@ public static class Builder {
private HRegionFileSystem regionFileSystem;
private CellComparator comparator;
private BloomType bloomFilterType;
+ private BloomFilterImpl bloomFilterImpl;
private Supplier> compactedFilesSupplier;
private Supplier favoredNodesSupplier;
private ColumnFamilyDescriptor family;
@@ -182,6 +189,11 @@ public Builder withBloomType(BloomType bloomFilterType) {
return this;
}
+ public Builder withBloomFilterImpl(BloomFilterImpl bloomFilterImpl) {
+ this.bloomFilterImpl = bloomFilterImpl;
+ return this;
+ }
+
public Builder
withCompactedFilesSupplier(Supplier> compactedFilesSupplier) {
this.compactedFilesSupplier = compactedFilesSupplier;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
index c2f3705fa631..93d89f91e9eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY;
+import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_IMPL_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_PARAM_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.COMPACTION_EVENT_KEY;
@@ -98,6 +99,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
private final Configuration conf;
private final CacheConfig cacheConf;
private final BloomType bloomType;
+ private final BloomFilterImpl bloomFilterImpl;
private final long maxKeys;
private final InetSocketAddress[] favoredNodes;
private final HFileContext fileContext;
@@ -137,8 +139,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
* @throws IOException problem writing to FS
*/
private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePath,
- final Configuration conf, CacheConfig cacheConf, BloomType bloomType, long maxKeys,
- InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind,
+ final Configuration conf, CacheConfig cacheConf, BloomType bloomType,
+ BloomFilterImpl bloomFilterImpl, long maxKeys, InetSocketAddress[] favoredNodes,
+ HFileContext fileContext, boolean shouldDropCacheBehind,
Supplier> compactedFilesSupplier, CellComparator comparator,
int maxVersions, boolean newVersionBehavior) throws IOException {
this.fs = fs;
@@ -146,6 +149,7 @@ private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePat
this.conf = conf;
this.cacheConf = cacheConf;
this.bloomType = bloomType;
+ this.bloomFilterImpl = bloomFilterImpl;
this.maxKeys = maxKeys;
this.favoredNodes = favoredNodes;
this.fileContext = fileContext;
@@ -154,8 +158,9 @@ private StoreFileWriter(FileSystem fs, Path liveFilePath, Path historicalFilePat
this.comparator = comparator;
this.maxVersions = maxVersions;
this.newVersionBehavior = newVersionBehavior;
- liveFileWriter = new SingleStoreFileWriter(fs, liveFilePath, conf, cacheConf, bloomType,
- maxKeys, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
+ liveFileWriter =
+ new SingleStoreFileWriter(fs, liveFilePath, conf, cacheConf, bloomType, bloomFilterImpl,
+ maxKeys, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
}
public static boolean shouldEnableHistoricalCompactionFiles(Configuration conf) {
@@ -332,9 +337,9 @@ public static Path getUniqueFile(final FileSystem fs, final Path dir) throws IOE
private SingleStoreFileWriter getHistoricalFileWriter() throws IOException {
if (historicalFileWriter == null) {
- historicalFileWriter =
- new SingleStoreFileWriter(fs, historicalFilePath, conf, cacheConf, bloomType, maxKeys,
- favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier);
+ historicalFileWriter = new SingleStoreFileWriter(fs, historicalFilePath, conf, cacheConf,
+ bloomType, bloomFilterImpl, maxKeys, favoredNodes, fileContext, shouldDropCacheBehind,
+ compactedFilesSupplier);
}
return historicalFileWriter;
}
@@ -502,6 +507,7 @@ private static class SingleStoreFileWriter {
private final BloomFilterWriter generalBloomFilterWriter;
private final BloomFilterWriter deleteFamilyBloomFilterWriter;
private final BloomType bloomType;
+ private final BloomFilterImpl bloomFilterImpl;
private byte[] bloomParam = null;
private long deleteFamilyCnt = 0;
private BloomContext bloomContext = null;
@@ -525,8 +531,8 @@ private static class SingleStoreFileWriter {
* @throws IOException problem writing to FS
*/
private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf,
- CacheConfig cacheConf, BloomType bloomType, long maxKeys, InetSocketAddress[] favoredNodes,
- HFileContext fileContext, boolean shouldDropCacheBehind,
+ CacheConfig cacheConf, BloomType bloomType, BloomFilterImpl bloomFilterImpl, long maxKeys,
+ InetSocketAddress[] favoredNodes, HFileContext fileContext, boolean shouldDropCacheBehind,
Supplier> compactedFilesSupplier) throws IOException {
this.compactedFilesSupplier = compactedFilesSupplier;
// TODO : Change all writers to be specifically created for compaction context
@@ -534,8 +540,10 @@ private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf
HFile.getWriterFactory(conf, cacheConf).withPath(fs, path).withFavoredNodes(favoredNodes)
.withFileContext(fileContext).withShouldDropCacheBehind(shouldDropCacheBehind).create();
+ this.bloomFilterImpl = bloomFilterImpl;
+
generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(conf, cacheConf,
- bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
+ bloomType, bloomFilterImpl, writer);
if (generalBloomFilterWriter != null) {
this.bloomType = bloomType;
@@ -570,10 +578,10 @@ private SingleStoreFileWriter(FileSystem fs, Path path, final Configuration conf
this.bloomType = BloomType.NONE;
}
- // initialize delete family Bloom filter when there is NO RowCol Bloom filter
+ // initialize delete family Bloom filter when there is NO RowCol Bloom/Ribbon filter
if (this.bloomType != BloomType.ROWCOL) {
- this.deleteFamilyBloomFilterWriter = BloomFilterFactory.createDeleteBloomAtWrite(conf,
- cacheConf, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
+ this.deleteFamilyBloomFilterWriter =
+ BloomFilterFactory.createDeleteBloomAtWrite(conf, cacheConf, bloomFilterImpl, writer);
deleteFamilyBloomContext =
new RowBloomContext(deleteFamilyBloomFilterWriter, fileContext.getCellComparator());
} else {
@@ -751,6 +759,7 @@ private boolean closeGeneralBloomFilter() throws IOException {
if (hasGeneralBloom) {
writer.addGeneralBloomFilter(generalBloomFilterWriter);
writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
+ writer.appendFileInfo(BLOOM_FILTER_IMPL_KEY, Bytes.toBytes(bloomFilterImpl.toString()));
if (bloomParam != null) {
writer.appendFileInfo(BLOOM_FILTER_PARAM_KEY, bloomParam);
}
@@ -810,6 +819,7 @@ public static class Builder {
private final FileSystem fs;
private BloomType bloomType = BloomType.NONE;
+ private BloomFilterImpl bloomFilterImpl = BloomFilterImpl.BLOOM;
private long maxKeyCount = 0;
private Path dir;
private Path liveFilePath;
@@ -884,6 +894,12 @@ public Builder withBloomType(BloomType bloomType) {
return this;
}
+ public Builder withBloomFilterImpl(BloomFilterImpl bloomFilterImpl) {
+ Preconditions.checkNotNull(bloomFilterImpl);
+ this.bloomFilterImpl = bloomFilterImpl;
+ return this;
+ }
+
/**
* @param maxKeyCount estimated maximum number of keys we expect to add
* @return this (for chained invocation)
@@ -1002,8 +1018,8 @@ public StoreFileWriter build() throws IOException {
}
}
return new StoreFileWriter(fs, liveFilePath, historicalFilePath, conf, cacheConf, bloomType,
- maxKeyCount, favoredNodes, fileContext, shouldDropCacheBehind, compactedFilesSupplier,
- comparator, maxVersions, newVersionBehavior);
+ bloomFilterImpl, maxKeyCount, favoredNodes, fileContext, shouldDropCacheBehind,
+ compactedFilesSupplier, comparator, maxVersions, newVersionBehavior);
}
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
index 87eca7b93c9c..5d2494ddad93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/storefiletracker/StoreFileTrackerBase.java
@@ -198,8 +198,9 @@ public final StoreFileWriter createWriter(CreateStoreFileWriterParams params) th
StoreFileWriter.Builder builder =
new StoreFileWriter.Builder(conf, writerCacheConf, ctx.getRegionFileSystem().getFileSystem())
.withOutputDir(outputDir).withBloomType(ctx.getBloomFilterType())
- .withMaxKeyCount(params.maxKeyCount()).withFavoredNodes(ctx.getFavoredNodes())
- .withFileContext(hFileContext).withShouldDropCacheBehind(params.shouldDropBehind())
+ .withBloomFilterImpl(ctx.getBloomFilterImpl()).withMaxKeyCount(params.maxKeyCount())
+ .withFavoredNodes(ctx.getFavoredNodes()).withFileContext(hFileContext)
+ .withShouldDropCacheBehind(params.shouldDropBehind())
.withCompactedFilesSupplier(ctx.getCompactedFilesSupplier())
.withFileStoragePolicy(params.fileStoragePolicy())
.withWriterCreationTracker(params.writerCreationTracker())
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 ed0e51f84e2e..142c29b88ef5 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
@@ -20,14 +20,22 @@
import java.io.DataInput;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellComparatorImpl;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
+import org.apache.hadoop.hbase.io.hfile.CompoundRibbonFilter;
+import org.apache.hadoop.hbase.io.hfile.CompoundRibbonFilterBase;
+import org.apache.hadoop.hbase.io.hfile.CompoundRibbonFilterWriter;
import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.regionserver.BloomFilterImpl;
import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.ribbon.RibbonFilterUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -56,11 +64,6 @@ private BloomFilterFactory() {
*/
public static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
- /**
- * For default (single-block) Bloom filters this specifies the maximum number of keys.
- */
- public static final String IO_STOREFILE_BLOOM_MAX_KEYS = "io.storefile.bloom.max.keys";
-
/** Master switch to enable Bloom filters */
public static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
@@ -74,6 +77,12 @@ private BloomFilterFactory() {
*/
public static final String IO_STOREFILE_BLOOM_BLOCK_SIZE = "io.storefile.bloom.block.size";
+ /**
+ * Default filter implementation to use. Can be "BLOOM" or "RIBBON" (case insensitive). This
+ * serves as the default when a column family does not explicitly specify a filter implementation.
+ */
+ public static final String IO_STOREFILE_BLOOM_FILTER_IMPL = "io.storefile.bloom.filter.impl";
+
/** Maximum number of times a Bloom filter can be "folded" if oversized */
private static final int MAX_ALLOWED_FOLD_FACTOR = 7;
@@ -92,13 +101,11 @@ public static BloomFilter createFromMeta(DataInput meta, HFile.Reader reader)
public static BloomFilter createFromMeta(DataInput meta, HFile.Reader reader,
BloomFilterMetrics metrics) throws IllegalArgumentException, IOException {
int version = meta.readInt();
- switch (version) {
- case CompoundBloomFilterBase.VERSION:
- return new CompoundBloomFilter(meta, reader, metrics);
-
- default:
- throw new IllegalArgumentException("Bad bloom filter format version " + version);
- }
+ return switch (version) {
+ case CompoundBloomFilterBase.VERSION -> new CompoundBloomFilter(meta, reader, metrics);
+ case CompoundRibbonFilterBase.VERSION -> new CompoundRibbonFilter(meta, reader, metrics);
+ default -> throw new IllegalArgumentException("Bad bloom filter format version " + version);
+ };
}
/**
@@ -118,6 +125,20 @@ public static float getErrorRate(Configuration conf) {
return conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float) 0.01);
}
+ /**
+ * Returns the adjusted error rate for the given bloom type. In case of row/column bloom filter
+ * lookups, each lookup is an OR of two separate lookups. Therefore, if each lookup's false
+ * positive rate is p, the resulting false positive rate is err = 1 - (1 - p)^2, and p = 1 -
+ * sqrt(1 - err).
+ */
+ private static double getAdjustedErrorRate(Configuration conf, BloomType bloomType) {
+ double err = getErrorRate(conf);
+ if (bloomType == BloomType.ROWCOL) {
+ err = 1 - Math.sqrt(1 - err);
+ }
+ return err;
+ }
+
/** Returns 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);
@@ -128,22 +149,41 @@ public static int getBloomBlockSize(Configuration conf) {
return conf.getInt(IO_STOREFILE_BLOOM_BLOCK_SIZE, 128 * 1024);
}
- /** Returns 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);
+ /**
+ * Returns the filter implementation for a column family. If the column family has an explicitly
+ * set filter implementation, that value is used. Otherwise, falls back to the global
+ * configuration default. The value is case insensitive. Defaults to BLOOM if not specified.
+ * @param conf the configuration
+ * @param family the column family descriptor (may be null)
+ * @return the filter implementation to use
+ */
+ public static BloomFilterImpl getBloomFilterImpl(Configuration conf,
+ ColumnFamilyDescriptor family) {
+ // Check family-level setting first
+ if (family != null) {
+ String impl = family.getValue(ColumnFamilyDescriptorBuilder.BLOOMFILTER_IMPL);
+ if (impl != null) {
+ return BloomFilterImpl.valueOf(impl.toUpperCase());
+ }
+ }
+ // Fall back to global configuration
+ String impl = conf.get(IO_STOREFILE_BLOOM_FILTER_IMPL);
+ if (impl == null) {
+ return BloomFilterImpl.BLOOM;
+ }
+ return BloomFilterImpl.valueOf(impl.toUpperCase());
}
/**
- * Creates a new general (Row or RowCol) Bloom filter at the time of
+ * Creates a new general (Row or RowCol) Bloom or Ribbon filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
- * @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
- * @return the new Bloom filter, or null in case Bloom filters are disabled or when failed to
+ * @param bloomImpl The filter implementation (BLOOM or RIBBON)
+ * @param writer the HFile writer
+ * @return the new Bloom/Ribbon filter, or null in case filters are disabled or when failed to
* create one.
*/
public static BloomFilterWriter createGeneralBloomAtWrite(Configuration conf,
- CacheConfig cacheConf, BloomType bloomType, int maxKeys, HFile.Writer writer) {
+ CacheConfig cacheConf, BloomType bloomType, BloomFilterImpl bloomImpl, HFile.Writer writer) {
if (!isGeneralBloomEnabled(conf)) {
LOG.trace("Bloom filters are disabled by configuration for " + writer.getPath()
+ (conf == null ? " (configuration is null)" : ""));
@@ -153,16 +193,12 @@ public static BloomFilterWriter createGeneralBloomAtWrite(Configuration conf,
return null;
}
- float err = getErrorRate(conf);
-
- // In case of row/column Bloom filter lookups, each lookup is an OR if two
- // separate lookups. Therefore, if each lookup's false positive rate is p,
- // the resulting false positive rate is err = 1 - (1 - p)^2, and
- // p = 1 - sqrt(1 - err).
- if (bloomType == BloomType.ROWCOL) {
- err = (float) (1 - Math.sqrt(1 - err));
+ // Check if Ribbon filter is requested
+ if (bloomImpl == BloomFilterImpl.RIBBON) {
+ return createRibbonFilterAtWrite(conf, cacheConf, bloomType, writer);
}
+ float err = (float) getAdjustedErrorRate(conf, bloomType);
int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, MAX_ALLOWED_FOLD_FACTOR);
// Do we support compound bloom filters?
@@ -175,30 +211,97 @@ public static BloomFilterWriter createGeneralBloomAtWrite(Configuration conf,
}
/**
- * Creates a new Delete Family Bloom filter at the time of
+ * Creates a new Ribbon filter at the time of HStoreFile writing.
+ * @param conf Configuration
+ * @param cacheConf Cache configuration
+ * @param bloomType The bloom type for key extraction (ROW, ROWCOL, etc.)
+ * @param writer The HFile writer
+ * @return The new Ribbon filter writer
+ */
+ private static BloomFilterWriter createRibbonFilterAtWrite(Configuration conf,
+ CacheConfig cacheConf, BloomType bloomType, HFile.Writer writer) {
+ int blockSize = getBloomBlockSize(conf);
+ int hashType = Hash.getHashType(conf);
+ double fpRate = getAdjustedErrorRate(conf, bloomType);
+
+ CellComparator comparator =
+ bloomType == BloomType.ROWCOL ? CellComparatorImpl.COMPARATOR : null;
+
+ CompoundRibbonFilterWriter ribbonWriter =
+ new CompoundRibbonFilterWriter(blockSize, RibbonFilterUtil.DEFAULT_BANDWIDTH, hashType,
+ cacheConf.shouldCacheBloomsOnWrite(), comparator, bloomType, fpRate);
+
+ writer.addInlineBlockWriter(ribbonWriter);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Created Ribbon filter for {} with blockSize={}, fpRate={}", writer.getPath(),
+ blockSize, fpRate);
+ }
+
+ return ribbonWriter;
+ }
+
+ /**
+ * Creates a new Delete Family Bloom or Ribbon filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
- * @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
- * @return the new Bloom filter, or null in case Bloom filters are disabled or when failed to
+ *
+ * If the bloom filter implementation is RIBBON, the delete family filter will also use Ribbon.
+ * Otherwise, a traditional Bloom filter is created.
+ * @param conf Configuration
+ * @param cacheConf Cache configuration
+ * @param bloomImpl The filter implementation (BLOOM or RIBBON)
+ * @param writer the HFile writer
+ * @return the new Bloom/Ribbon filter, or null in case filters are disabled or when failed to
* create one.
*/
public static BloomFilterWriter createDeleteBloomAtWrite(Configuration conf,
- CacheConfig cacheConf, int maxKeys, HFile.Writer writer) {
+ CacheConfig cacheConf, BloomFilterImpl bloomImpl, 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);
+ // Use Ribbon filter if the implementation is Ribbon
+ if (bloomImpl == BloomFilterImpl.RIBBON) {
+ return createDeleteRibbonAtWrite(conf, cacheConf, writer);
+ }
+ // Use traditional Bloom filter
+ float err = getErrorRate(conf);
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(), null, BloomType.ROW);
writer.addInlineBlockWriter(bloomWriter);
return bloomWriter;
}
+
+ /**
+ * Creates a new Delete Family Ribbon filter at the time of HStoreFile writing.
+ * @param conf Configuration
+ * @param cacheConf Cache configuration
+ * @param writer The HFile writer
+ * @return The new Ribbon filter writer
+ */
+ private static BloomFilterWriter createDeleteRibbonAtWrite(Configuration conf,
+ CacheConfig cacheConf, HFile.Writer writer) {
+ int blockSize = getBloomBlockSize(conf);
+ int hashType = Hash.getHashType(conf);
+ double fpRate = getErrorRate(conf);
+
+ CompoundRibbonFilterWriter ribbonWriter =
+ new CompoundRibbonFilterWriter(blockSize, RibbonFilterUtil.DEFAULT_BANDWIDTH, hashType,
+ cacheConf.shouldCacheBloomsOnWrite(), null, BloomType.ROW, fpRate);
+
+ writer.addInlineBlockWriter(ribbonWriter);
+
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("Created Delete Family Ribbon filter for {} with blockSize={}, fpRate={}",
+ writer.getPath(), blockSize, fpRate);
+ }
+
+ return ribbonWriter;
+ }
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
index 5b24a2714747..9fe9f1d5e6af 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterUtil.java
@@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hbase.util;
-import static org.apache.hadoop.hbase.regionserver.BloomType.ROWPREFIX_FIXED_LENGTH;
-
import java.text.NumberFormat;
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
@@ -255,7 +253,7 @@ public static byte[] getBloomFilterParam(BloomType bloomFilterType, Configuratio
throws IllegalArgumentException {
byte[] bloomParam = null;
String message = "Bloom filter type is " + bloomFilterType + ", ";
- if (bloomFilterType.equals(ROWPREFIX_FIXED_LENGTH)) {
+ if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) {
String prefixLengthString = conf.get(PREFIX_LENGTH_KEY);
if (prefixLengthString == null) {
message += PREFIX_LENGTH_KEY + " not specified.";
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/InterleavedRibbonSolution.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/InterleavedRibbonSolution.java
new file mode 100644
index 000000000000..607b1a572070
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/InterleavedRibbonSolution.java
@@ -0,0 +1,328 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.nio.ByteBuff;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Interleaved Column-Major Layout (ICML) storage for Ribbon Filter solution.
+ *
+ * This layout provides better space efficiency by supporting fractional bits per key. Row-major
+ * layout requires fixed power-of-2 byte sizes (1, 2, 4, or 8 bytes) per slot, but ICML stores
+ * variable columns (bits) per slot, allowing precise FPR targeting.
+ *
+ * Memory Layout:
+ *
+ *
+ * - Data is divided into blocks of 64 slots
+ * - Each block contains numColumns segments (64-bit values)
+ * - Blocks before upperStartBlock use (upperNumColumns - 1) columns
+ * - Blocks from upperStartBlock onwards use upperNumColumns columns
+ *
+ * Example for 7.5 bits/key average (upperStartBlock = 2):
+ * Block 0: 7 columns
+ * Block 1: 7 columns
+ * Block 2: 8 columns (upperStartBlock)
+ * Block 3: 8 columns
+ * ...
+ *
+ */
+@InterfaceAudience.Private
+public class InterleavedRibbonSolution {
+
+ /** Number of bits in a coefficient row (block size in slots). */
+ public static final int COEFF_BITS = RibbonFilterUtil.DEFAULT_BANDWIDTH;
+
+ /** Bytes per segment (sizeof(long) = 8 bytes for 64-bit bandwidth) */
+ public static final int SEGMENT_BYTES = COEFF_BITS / 8;
+
+ /** The segment data (column-major within blocks) */
+ private final long[] segments;
+
+ /** Total number of slots (must be multiple of COEFF_BITS) */
+ private final int numSlots;
+
+ /** Number of blocks */
+ private final int numBlocks;
+
+ /** Upper number of columns (some blocks use this) */
+ private final int upperNumColumns;
+
+ /** Block index from which upperNumColumns is used (blocks before use lowerNumColumns) */
+ private final int upperStartBlock;
+
+ /** Cached ByteBuff view of segments for instance contains method (used in tests) */
+ private ByteBuff segmentsBuff;
+
+ /**
+ * Creates an InterleavedRibbonSolution with specified parameters.
+ * @param numSlots Number of slots (will be rounded up to multiple of COEFF_BITS)
+ * @param desiredFpRate Desired false positive rate (e.g., 0.01 for 1%)
+ */
+ public InterleavedRibbonSolution(int numSlots, double desiredFpRate) {
+ // Round up to multiple of COEFF_BITS
+ this.numSlots = roundUpNumSlots(numSlots);
+ this.numBlocks = this.numSlots / COEFF_BITS;
+
+ // Calculate columns needed for desired FP rate
+ // FPR = 2^(-numColumns), so numColumns = -log2(FPR) = log2(1/FPR)
+ double oneInFpRate = 1.0 / desiredFpRate;
+
+ if (oneInFpRate <= 1.0 || numBlocks == 0) {
+ // Edge case: 100% FP rate or empty
+ this.upperNumColumns = 1;
+ this.upperStartBlock = 0;
+ this.segments = new long[numBlocks];
+ } else {
+ // Calculate lower and upper column counts for fractional bits
+ int lowerColumns = floorLog2((long) oneInFpRate);
+ double lowerFpRate = Math.pow(2.0, -lowerColumns);
+ double upperFpRate = Math.pow(2.0, -(lowerColumns + 1));
+
+ // Proportion of slots using lower columns
+ double lowerPortion = (desiredFpRate - upperFpRate) / (lowerFpRate - upperFpRate);
+ lowerPortion = Math.max(0.0, Math.min(1.0, lowerPortion));
+
+ // Calculate upper_start_block
+ int numStarts = this.numSlots - COEFF_BITS + 1;
+ this.upperStartBlock = (int) ((lowerPortion * numStarts) / COEFF_BITS);
+ this.upperNumColumns = lowerColumns + 1;
+
+ // Calculate total segments needed
+ // Blocks [0, upperStartBlock) use lowerColumns
+ // Blocks [upperStartBlock, numBlocks) use upperNumColumns
+ int numSegments =
+ upperStartBlock * lowerColumns + (numBlocks - upperStartBlock) * upperNumColumns;
+ this.segments = new long[numSegments];
+ }
+ }
+
+ /**
+ * Rounds up to a number of slots supported by this structure.
+ */
+ public static int roundUpNumSlots(int numSlots) {
+ // Must be multiple of COEFF_BITS, minimum 2 * COEFF_BITS
+ int rounded = ((numSlots + COEFF_BITS - 1) / COEFF_BITS) * COEFF_BITS;
+ return Math.max(rounded, 2 * COEFF_BITS);
+ }
+
+ /**
+ * Returns the number of columns for a given block.
+ */
+ public int getNumColumns(int blockIndex) {
+ return (blockIndex < upperStartBlock) ? (upperNumColumns - 1) : upperNumColumns;
+ }
+
+ /**
+ * Performs back substitution from a RibbonBanding to populate this solution.
+ * @param banding The completed banding storage
+ */
+ public void backSubstFrom(RibbonBanding banding) {
+ // State buffer: stores last COEFF_BITS solution values per column
+ long[] state = new long[upperNumColumns];
+
+ int segmentNum = segments.length;
+
+ // Process blocks from end to start
+ for (int block = numBlocks - 1; block >= 0; block--) {
+ int startSlot = block * COEFF_BITS;
+ int blockColumns = getNumColumns(block);
+
+ // Process each slot in the block (reverse order within block)
+ for (int i = COEFF_BITS - 1; i >= 0; i--) {
+ int slotIndex = startSlot + i;
+ long cr = banding.getCoeffRow(slotIndex);
+ int rr;
+
+ // Handle empty rows with pseudorandom fill
+ if (cr == 0) {
+ // Pseudorandom fill for empty rows
+ rr = (int) (slotIndex * 0x9E3779B185EBCA87L);
+ } else {
+ rr = 0;
+ }
+
+ // Compute solution for each column
+ for (int col = 0; col < blockColumns; col++) {
+ // Shift state left by 1 (make room for new bit at position 0)
+ long tmp = state[col] << 1;
+
+ // Compute next solution bit using parity
+ // bit = parity(tmp & cr) XOR ((rr >> col) & 1)
+ int bit = Long.bitCount(tmp & cr) & 1;
+ bit ^= (rr >> col) & 1;
+
+ // Store the bit
+ tmp |= bit;
+ state[col] = tmp;
+ }
+ }
+
+ // Write state to segments for this block
+ segmentNum -= blockColumns;
+ System.arraycopy(state, 0, segments, segmentNum, blockColumns);
+ }
+ }
+
+ /**
+ * Checks if a key is (probably) in the filter.
+ * @param start Starting position from hash
+ * @param coeffRow Coefficient row from hash
+ * @param expectedResult Expected result (always 0)
+ * @return true if the key might be in the filter, false if definitely not
+ */
+ public boolean contains(int start, long coeffRow, int expectedResult) {
+ if (segmentsBuff == null) {
+ ByteBuffer bb = ByteBuffer.allocate(segments.length * Long.BYTES);
+ bb.asLongBuffer().put(segments);
+ segmentsBuff = new SingleByteBuff(bb);
+ }
+ return contains(start, coeffRow, expectedResult, segmentsBuff, 0, numSlots, upperNumColumns,
+ upperStartBlock);
+ }
+
+ /**
+ * Static contains method for querying from ByteBuff. Used when reading directly from HFile
+ * blocks.
+ * @param start Starting position from hash
+ * @param coeffRow Coefficient row from hash
+ * @param expectedResult Expected result
+ * @param buf ByteBuff containing segment data
+ * @param offset Offset where segment data starts
+ * @param numSlots Number of slots
+ * @param upperNumColumns Upper column count
+ * @param upperStartBlock Block index where upper columns start
+ * @return true if key might be in filter
+ */
+ public static boolean contains(int start, long coeffRow, int expectedResult, ByteBuff buf,
+ int offset, int numSlots, int upperNumColumns, int upperStartBlock) {
+
+ int numBlocks = numSlots / COEFF_BITS;
+
+ if (start < 0 || start >= numSlots - COEFF_BITS + 1) {
+ return false;
+ }
+
+ int startBlock = start / COEFF_BITS;
+ int startBit = start % COEFF_BITS;
+
+ // Calculate segment position for this block
+ int segmentNum;
+ int numColumns;
+ if (startBlock < upperStartBlock) {
+ // Blocks before upperStartBlock use lowerNumColumns
+ segmentNum = startBlock * (upperNumColumns - 1);
+ numColumns = upperNumColumns - 1;
+ } else {
+ // Blocks from upperStartBlock onwards use upperNumColumns
+ segmentNum =
+ upperStartBlock * (upperNumColumns - 1) + (startBlock - upperStartBlock) * upperNumColumns;
+ numColumns = upperNumColumns;
+ }
+
+ // Split coeffRow
+ long crLeft = coeffRow << startBit;
+ long crRight = (startBit == 0) ? 0 : (coeffRow >>> (COEFF_BITS - startBit));
+
+ // Next block info
+ int nextBlockColumns = 0;
+ int nextBlockSegmentStart = 0;
+ if (startBit != 0 && startBlock + 1 < numBlocks) {
+ int nextBlock = startBlock + 1;
+ if (nextBlock < upperStartBlock) {
+ nextBlockColumns = upperNumColumns - 1;
+ nextBlockSegmentStart = nextBlock * (upperNumColumns - 1);
+ } else {
+ nextBlockColumns = upperNumColumns;
+ nextBlockSegmentStart =
+ upperStartBlock * (upperNumColumns - 1) + (nextBlock - upperStartBlock) * upperNumColumns;
+ }
+ }
+
+ int columnsToCheck =
+ (nextBlockColumns > 0) ? Math.min(numColumns, nextBlockColumns) : numColumns;
+
+ for (int col = 0; col < columnsToCheck; col++) {
+ // Load segment from buffer (little-endian)
+ int bufPos = offset + (segmentNum + col) * SEGMENT_BYTES;
+ long seg = buf.getLong(bufPos);
+ long solnData = seg & crLeft;
+
+ if (nextBlockColumns > 0) {
+ int nextBufPos = offset + (nextBlockSegmentStart + col) * SEGMENT_BYTES;
+ long nextSeg = buf.getLong(nextBufPos);
+ solnData |= nextSeg & crRight;
+ }
+
+ int bit = Long.bitCount(solnData) & 1;
+
+ if (((expectedResult >> col) & 1) != bit) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ /**
+ * Writes the solution to a DataOutput stream.
+ */
+ public void writeTo(DataOutput out) throws IOException {
+ out.writeInt(numSlots);
+ out.writeInt(upperNumColumns);
+ out.writeInt(upperStartBlock);
+ out.writeInt(segments.length);
+ for (long segment : segments) {
+ out.writeLong(segment);
+ }
+ }
+
+ /**
+ * Computes floor(log2(n)) for positive n.
+ */
+ private static int floorLog2(long n) {
+ if (n <= 0) {
+ return 0;
+ }
+ return 63 - Long.numberOfLeadingZeros(n);
+ }
+
+ /**
+ * Returns the size in bytes.
+ */
+ public long getByteSize() {
+ return (long) segments.length * SEGMENT_BYTES;
+ }
+
+ public int getUpperNumColumns() {
+ return upperNumColumns;
+ }
+
+ public int getUpperStartBlock() {
+ return upperStartBlock;
+ }
+
+ public long[] getSegments() {
+ return segments;
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonBanding.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonBanding.java
new file mode 100644
index 000000000000..ad5188a75216
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonBanding.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Implements the banding phase of Homogeneous Ribbon Filter construction using Gaussian elimination
+ * in GF(2).
+ *
+ * The banding process takes key hash results (start, coeffRow) and builds a banded matrix that can
+ * later be solved via back substitution to produce the filter solution. Construction never fails -
+ * inconsistencies are absorbed and empty rows are filled with pseudorandom data.
+ *
+ * This implementation is optimized for Homogeneous Ribbon Filter where resultRow is always 0,
+ * eliminating the need for result row storage.
+ * @see Ribbon Filter Paper
+ */
+@InterfaceAudience.Private
+public class RibbonBanding {
+
+ /** Coefficient rows storage */
+ private final long[] coeffRows;
+
+ /** Number of slots */
+ private final int numSlots;
+
+ /** Bandwidth */
+ private final int bandwidth;
+
+ /** Number of keys successfully added */
+ private int numAdded;
+
+ /**
+ * Creates a new RibbonBanding storage.
+ * @param numSlots Number of slots in the ribbon filter
+ * @param bandwidth Coefficient width in bits (typically 64)
+ */
+ public RibbonBanding(int numSlots, int bandwidth) {
+ if (numSlots <= 0) {
+ throw new IllegalArgumentException("numSlots must be positive: " + numSlots);
+ }
+ RibbonFilterUtil.validateBandwidth(bandwidth);
+
+ this.numSlots = numSlots;
+ this.bandwidth = bandwidth;
+ this.coeffRows = new long[numSlots];
+ this.numAdded = 0;
+ }
+
+ /**
+ * Adds a single entry to the banding storage using Gaussian elimination. This method always
+ * succeeds because inconsistencies are absorbed during back substitution.
+ *
+ * For Homogeneous Ribbon Filter, resultRow is always 0 and not stored.
+ * @param start Starting row position
+ * @param coeffRow Coefficient row
+ */
+ public void add(int start, long coeffRow) {
+ if (start < 0 || start >= numSlots) {
+ throw new IllegalArgumentException(
+ "start position out of range: " + start + " (numSlots=" + numSlots + ")");
+ }
+
+ int i = start;
+ long cr = coeffRow;
+
+ while (true) {
+ // Check bounds
+ if (i >= numSlots) {
+ // Coefficient row extends beyond available slots
+ // This is absorbed (never fails)
+ return;
+ }
+
+ long existingCr = coeffRows[i];
+
+ if (existingCr == 0) {
+ // Empty slot found - store the coefficient row
+ coeffRows[i] = cr;
+ numAdded++;
+ return;
+ }
+
+ // Gaussian elimination: XOR with existing row
+ cr ^= existingCr;
+
+ if (cr == 0) {
+ // Coefficient row became zero
+ // Inconsistency is absorbed (always succeeds)
+ return;
+ }
+
+ // Move to next position based on trailing zeros
+ int tz = Long.numberOfTrailingZeros(cr);
+ i += tz;
+ cr >>>= tz;
+ }
+ }
+
+ /**
+ * Returns the number of keys successfully added.
+ */
+ public int getNumAdded() {
+ return numAdded;
+ }
+
+ /**
+ * Returns the number of slots.
+ */
+ public int getNumSlots() {
+ return numSlots;
+ }
+
+ /**
+ * Returns the bandwidth.
+ */
+ public int getBandwidth() {
+ return bandwidth;
+ }
+
+ /**
+ * Returns the coefficient row at the given index (for testing/debugging).
+ */
+ long getCoeffRow(int index) {
+ return coeffRows[index];
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterChunk.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterChunk.java
new file mode 100644
index 000000000000..ee9c4d0337a1
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterChunk.java
@@ -0,0 +1,292 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.BloomFilterBase;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A single chunk of a Ribbon Filter using ICML storage.
+ * @see InterleavedRibbonSolution
+ */
+@InterfaceAudience.Private
+public class RibbonFilterChunk implements BloomFilterBase {
+
+ /** Bandwidth (coefficient width in bits) */
+ private final int bandwidth;
+
+ /** Hash type */
+ private final int hashType;
+
+ /** Bloom type for key extraction */
+ private final BloomType bloomType;
+
+ /** Desired false positive rate */
+ private final double desiredFpRate;
+
+ /** Pre-calculated overhead ratio (passed from higher level) */
+ private final double overheadRatio;
+
+ /** Number of keys added */
+ private int keyCount;
+
+ /** Maximum number of keys (capacity) */
+ private int maxKeys;
+
+ /** Number of slots */
+ private int numSlots;
+
+ /** The hasher (created during finalization) */
+ private RibbonHasher hasher;
+
+ /** The interleaved solution (created during finalization) */
+ private InterleavedRibbonSolution interleavedSolution;
+
+ /**
+ * Key buffer for lazy allocation. Keys are buffered during add() and processed during
+ * finalizeRibbon() to allow optimal sizing based on actual key count.
+ */
+ private List keyBuffer;
+
+ /**
+ * Creates a new RibbonFilterChunk for writing with ICML storage mode.
+ * @param bandwidth Bandwidth (typically 64)
+ * @param hashType Hash type
+ * @param bloomType Bloom type for key extraction
+ * @param desiredFpRate Desired false positive rate (e.g., 0.01 for 1%)
+ * @param overheadRatio Pre-calculated overhead ratio
+ */
+ public RibbonFilterChunk(int bandwidth, int hashType, BloomType bloomType, double desiredFpRate,
+ double overheadRatio) {
+ RibbonFilterUtil.validateBandwidth(bandwidth);
+
+ this.bandwidth = bandwidth;
+ this.hashType = hashType;
+ this.bloomType = bloomType;
+ this.desiredFpRate = desiredFpRate;
+ this.overheadRatio = overheadRatio;
+ this.keyCount = 0;
+ this.maxKeys = 0;
+ this.numSlots = 0;
+ }
+
+ /**
+ * Creates a new RibbonFilterChunk with default settings. Only used for testing.
+ *
+ * Calculates optimal overhead automatically.
+ * @param bloomType Bloom type for key extraction
+ */
+ public RibbonFilterChunk(BloomType bloomType) {
+ // Default 1% FP rate
+ this(RibbonFilterUtil.DEFAULT_BANDWIDTH, RibbonFilterUtil.getDefaultHashType(), bloomType, 0.01,
+ RibbonFilterUtil.computeOptimalOverheadForFpRate(0.01, RibbonFilterUtil.DEFAULT_BANDWIDTH));
+ }
+
+ /**
+ * Initializes the key buffer for lazy allocation. The actual banding matrix is allocated during
+ * finalization based on the actual number of keys added.
+ *
+ * This approach ensures optimal space usage by sizing the filter based on actual key count rather
+ * than estimated maximum keys.
+ * @param maxKeys Maximum number of keys expected (used for initial buffer capacity hint)
+ * @throws IllegalStateException If already allocated
+ */
+ public void allocRibbon(int maxKeys) {
+ if (keyBuffer != null) {
+ throw new IllegalStateException("Ribbon filter already allocated");
+ }
+
+ this.maxKeys = maxKeys;
+
+ // Initialize key buffer with reasonable initial capacity
+ // Use smaller of maxKeys or 1024 to avoid over-allocation for small chunks
+ int initialCapacity = Math.min(maxKeys, 1024);
+ keyBuffer = new ArrayList<>(Math.max(initialCapacity, 16));
+ }
+
+ /**
+ * Adds a Cell to the filter. The key is extracted and buffered for later processing during
+ * finalization.
+ * @param cell The cell to add
+ * @throws IllegalStateException If not allocated or already finalized
+ */
+ public void add(Cell cell) {
+ byte[] key = RibbonFilterUtil.extractKeyFromCell(cell, bloomType);
+ addKey(key);
+ }
+
+ /**
+ * Adds a raw key to the filter. The key is buffered for later processing during finalization.
+ * @param key The key bytes
+ * @throws IllegalStateException If not allocated or already finalized
+ */
+ public void addKey(byte[] key) {
+ if (keyBuffer == null) {
+ throw new IllegalStateException("Ribbon filter not allocated. Call allocRibbon() first.");
+ }
+ if (interleavedSolution != null) {
+ throw new IllegalStateException("Ribbon filter already finalized.");
+ }
+
+ keyBuffer.add(key);
+ keyCount++;
+ }
+
+ /**
+ * Finalizes the Ribbon filter by allocating optimal-sized structures and performing
+ * back-substitution.
+ *
+ * This method implements lazy allocation: the banding matrix is sized based on the actual number
+ * of keys added (keyCount) rather than the estimated maximum (maxKeys). This ensures optimal
+ * space usage, especially for chunks with fewer keys than expected.
+ * @throws IllegalStateException If not allocated or already finalized
+ */
+ public void finalizeRibbon() {
+ if (keyBuffer == null) {
+ throw new IllegalStateException("Ribbon filter not allocated. Call allocRibbon() first.");
+ }
+ if (interleavedSolution != null) {
+ throw new IllegalStateException("Ribbon filter already finalized.");
+ }
+
+ // Calculate optimal numSlots based on actual keyCount (not maxKeys)
+ // roundUpNumSlots() ensures minimum of 2*COEFF_BITS (128) slots
+ numSlots = InterleavedRibbonSolution
+ .roundUpNumSlots(RibbonFilterUtil.computeNumSlots(keyCount, overheadRatio));
+
+ // Now allocate hasher and banding with optimal size
+ hasher = new RibbonHasher(numSlots, bandwidth, hashType);
+ RibbonBanding banding = new RibbonBanding(numSlots, bandwidth);
+
+ // Add all buffered keys to the banding matrix
+ for (byte[] key : keyBuffer) {
+ RibbonHasher.RibbonHashResult hashResult = hasher.hash(key, 0, key.length);
+ banding.add(hashResult.start(), hashResult.coeffRow());
+ }
+
+ // Clear key buffer to free memory
+ keyBuffer = null;
+
+ // Compute solution via back substitution using ICML storage
+ double fpRate = desiredFpRate > 0 ? desiredFpRate : 0.01;
+ interleavedSolution = new InterleavedRibbonSolution(numSlots, fpRate);
+ interleavedSolution.backSubstFrom(banding);
+ }
+
+ /**
+ * Checks if a key is (probably) in the filter.
+ * @param key Key bytes
+ * @param offset Offset into key array
+ * @param length Length of key
+ * @return true if the key might be in the filter, false if definitely not
+ * @throws IllegalStateException If not finalized
+ */
+ public boolean contains(byte[] key, int offset, int length) {
+ if (interleavedSolution == null) {
+ throw new IllegalStateException("Ribbon filter not finalized. Call finalizeRibbon() first.");
+ }
+
+ RibbonHasher.RibbonHashResult hashResult = hasher.hash(key, offset, length);
+ return interleavedSolution.contains(hashResult.start(), hashResult.coeffRow(),
+ hashResult.resultRow());
+ }
+
+ /**
+ * Writes the Ribbon filter to a DataOutput stream. Only writes the solution data (not metadata).
+ * @param out The output stream
+ * @throws IOException If an I/O error occurs
+ * @throws IllegalStateException If not finalized
+ */
+ public void writeRibbon(DataOutput out) throws IOException {
+ if (interleavedSolution == null) {
+ throw new IllegalStateException("Ribbon filter not finalized. Call finalizeRibbon() first.");
+ }
+
+ // Write ICML segments (metadata is stored separately in MetaWriter)
+ long[] segments = interleavedSolution.getSegments();
+ for (long segment : segments) {
+ out.writeLong(segment);
+ }
+ }
+
+ /**
+ * Creates a new RibbonFilterChunk with the same configuration.
+ * @return A new chunk with the same settings
+ */
+ public RibbonFilterChunk createAnother() {
+ return new RibbonFilterChunk(bandwidth, hashType, bloomType, desiredFpRate, overheadRatio);
+ }
+
+ // BloomFilterBase implementation
+
+ @Override
+ public long getByteSize() {
+ if (interleavedSolution != null) {
+ return interleavedSolution.getByteSize();
+ }
+ return 0;
+ }
+
+ @Override
+ public long getKeyCount() {
+ return keyCount;
+ }
+
+ @Override
+ public long getMaxKeys() {
+ return maxKeys;
+ }
+
+ // Getters
+
+ public int getBandwidth() {
+ return bandwidth;
+ }
+
+ public int getHashType() {
+ return hashType;
+ }
+
+ public BloomType getBloomType() {
+ return bloomType;
+ }
+
+ public int getNumSlots() {
+ return numSlots;
+ }
+
+ /**
+ * Returns the interleaved solution.
+ * @return The interleaved solution, or null if not finalized
+ */
+ public InterleavedRibbonSolution getInterleavedSolution() {
+ return interleavedSolution;
+ }
+
+ @Override
+ public String toString() {
+ return RibbonFilterUtil.formatStats(keyCount, numSlots, bandwidth);
+ }
+
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterUtil.java
new file mode 100644
index 000000000000..c24cf9b8dc9f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonFilterUtil.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.util.Hash;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Utility methods for Ribbon Filter.
+ */
+@InterfaceAudience.Private
+public final class RibbonFilterUtil {
+
+ /** Default bandwidth (coefficient width in bits). */
+ public static final int DEFAULT_BANDWIDTH = 64;
+
+ /** Minimum overhead ratio */
+ public static final double MIN_OVERHEAD_RATIO = 0.01;
+
+ /** Maximum overhead ratio */
+ public static final double MAX_OVERHEAD_RATIO = 0.50;
+
+ /** Separator for stats formatting */
+ public static final String STATS_RECORD_SEP = "; ";
+
+ private RibbonFilterUtil() {
+ // Utility class, no instantiation
+ }
+
+ /**
+ * Computes the optimal number of slots for a Ribbon filter.
+ *
+ * For n keys with overhead ratio ε, the number of slots is: m = n * (1 + ε)
+ * @param numKeys Number of keys to store
+ * @param overheadRatio Space overhead ratio (e.g., 0.05 for 5%)
+ * @return Number of slots needed
+ */
+ public static int computeNumSlots(int numKeys, double overheadRatio) {
+ if (numKeys <= 0) {
+ return 0;
+ }
+ if (overheadRatio < MIN_OVERHEAD_RATIO) {
+ overheadRatio = MIN_OVERHEAD_RATIO;
+ }
+ if (overheadRatio > MAX_OVERHEAD_RATIO) {
+ overheadRatio = MAX_OVERHEAD_RATIO;
+ }
+
+ // m = n * (1 + overhead)
+ long slots = (long) Math.ceil(numKeys * (1.0 + overheadRatio));
+
+ // Ensure at least bandwidth slots for proper functioning
+ slots = Math.max(slots, DEFAULT_BANDWIDTH);
+
+ return (int) Math.min(slots, Integer.MAX_VALUE);
+ }
+
+ /**
+ * Computes the optimal slot overhead for Ribbon Filter using the formula: ε = (4 + r/4) / w,
+ * where r = -log₂(fpRate) and w = bandwidth.
+ * @param desiredFpRate Desired false positive rate (e.g., 0.01 for 1%)
+ * @param bandwidth Coefficient width in bits, typically 64
+ * @return Optimal overhead ratio
+ * @see Ribbon Filter Paper, Equation 7
+ */
+ public static double computeOptimalOverheadForFpRate(double desiredFpRate, int bandwidth) {
+ if (desiredFpRate <= 0 || desiredFpRate >= 1.0) {
+ throw new IllegalArgumentException("desiredFpRate must be in (0, 1): " + desiredFpRate);
+ }
+
+ // r = -log₂(fpRate)
+ double resultBits = -Math.log(desiredFpRate) / Math.log(2.0);
+
+ // ε = (4 + r/4) / w
+ double overhead = (4.0 + resultBits / 4.0) / bandwidth;
+
+ // Clamp to valid range
+ return Math.max(MIN_OVERHEAD_RATIO, Math.min(MAX_OVERHEAD_RATIO, overhead));
+ }
+
+ /**
+ * Computes the size in bytes of a Ribbon filter solution.
+ * @param numSlots Number of slots
+ * @return Size in bytes (1 byte per slot)
+ */
+ public static long computeByteSize(int numSlots) {
+ return numSlots; // 1 byte per slot
+ }
+
+ /**
+ * Formats statistics about a Ribbon filter for display.
+ * @param numKeys Number of keys stored
+ * @param numSlots Number of slots
+ * @param bandwidth Bandwidth
+ * @return Formatted statistics string
+ */
+ public static String formatStats(int numKeys, int numSlots, int bandwidth) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Ribbon Filter");
+ sb.append(STATS_RECORD_SEP);
+ sb.append("Keys: ").append(numKeys);
+ sb.append(STATS_RECORD_SEP);
+ sb.append("Slots: ").append(numSlots);
+ sb.append(STATS_RECORD_SEP);
+ sb.append("Bandwidth: ").append(bandwidth);
+ sb.append(STATS_RECORD_SEP);
+ sb.append("Byte size: ").append(computeByteSize(numSlots));
+
+ if (numKeys > 0) {
+ double overhead = (double) numSlots / numKeys - 1.0;
+ sb.append(STATS_RECORD_SEP);
+ sb.append(String.format("Overhead: %.2f%%", overhead * 100));
+ }
+
+ return sb.toString();
+ }
+
+ /**
+ * Validates the bandwidth parameter.
+ * @param bandwidth Bandwidth to validate
+ * @throws IllegalArgumentException If bandwidth is not {@link #DEFAULT_BANDWIDTH}
+ */
+ public static void validateBandwidth(int bandwidth) {
+ if (bandwidth != DEFAULT_BANDWIDTH) {
+ throw new IllegalArgumentException(
+ "Unsupported bandwidth: " + bandwidth + ". Only " + DEFAULT_BANDWIDTH + " is supported.");
+ }
+ }
+
+ /**
+ * Returns the default hash type for Ribbon filter.
+ */
+ public static int getDefaultHashType() {
+ return Hash.MURMUR_HASH3;
+ }
+
+ /**
+ * Computes the number of columns (bits per key) needed for a target false positive rate.
+ * @param fpRate Target false positive rate (e.g., 0.01 for 1%)
+ * @return Number of columns (minimum 1)
+ */
+ public static int computeFingerprintBits(double fpRate) {
+ if (fpRate <= 0 || fpRate >= 1.0) {
+ return 1;
+ }
+ return Math.max(1, (int) Math.ceil(-Math.log(fpRate) / Math.log(2.0)));
+ }
+
+ /**
+ * Extracts the key bytes from a Cell based on the bloom type.
+ * @param cell The cell to extract key from
+ * @param baseType The base bloom type (ROW or ROWCOL)
+ * @return The extracted key bytes
+ */
+ public static byte[] extractKeyFromCell(Cell cell, BloomType baseType) {
+ if (baseType == BloomType.ROWCOL) {
+ return PrivateCellUtil
+ .getCellKeySerializedAsKeyValueKey(PrivateCellUtil.createFirstOnRowCol(cell));
+ } else {
+ return CellUtil.copyRow(cell);
+ }
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonHasher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonHasher.java
new file mode 100644
index 000000000000..7a8149c27714
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/RibbonHasher.java
@@ -0,0 +1,198 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import org.apache.hadoop.hbase.util.ByteArrayHashKey;
+import org.apache.hadoop.hbase.util.Hash;
+import org.apache.hadoop.hbase.util.HashKey;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Hasher for Ribbon Filter that generates hash components needed for banding and querying.
+ *
+ * For each key, this class generates:
+ *
+ * - start: The starting row position in the band matrix
+ * - coeffRow: The coefficient row (w bits wide, where w is the bandwidth)
+ * - resultRow: Always 0 (no fingerprint storage)
+ *
+ *
+ * The hash function uses a 64-bit hash split into components for start position and coefficient
+ * generation.
+ */
+@InterfaceAudience.Private
+public class RibbonHasher {
+
+ /** Golden ratio constant for better hash mixing */
+ private static final long GOLDEN_RATIO = 0x9E3779B97F4A7C15L;
+
+ /** Number of slots in the ribbon filter */
+ private final int numSlots;
+
+ /** Bandwidth (coefficient width in bits, typically 64) */
+ private final int bandwidth;
+
+ /** Hash function instance */
+ private final Hash hash;
+
+ /** Hash type (stored for reference) */
+ private final int hashType;
+
+ /**
+ * Creates a new RibbonHasher.
+ * @param numSlots Number of slots in the ribbon filter (m)
+ * @param bandwidth Coefficient width in bits (w), typically 64
+ * @param hashType Hash type to use (e.g., Hash.MURMUR_HASH3)
+ */
+ public RibbonHasher(int numSlots, int bandwidth, int hashType) {
+ if (numSlots <= 0) {
+ throw new IllegalArgumentException("numSlots must be positive: " + numSlots);
+ }
+ RibbonFilterUtil.validateBandwidth(bandwidth);
+
+ this.numSlots = numSlots;
+ this.bandwidth = bandwidth;
+ this.hash = Hash.getInstance(hashType);
+ this.hashType = hashType;
+
+ if (this.hash == null) {
+ throw new IllegalArgumentException("Invalid hash type: " + hashType);
+ }
+ }
+
+ /**
+ * Computes hash components for a byte array key.
+ * @param key Key bytes
+ * @param offset Offset into key array
+ * @param length Length of key
+ * @return RibbonHashResult containing start, coeffRow, and resultRow
+ */
+ public RibbonHashResult hash(byte[] key, int offset, int length) {
+ HashKey hashKey = new ByteArrayHashKey(key, offset, length);
+ return computeHash(hashKey);
+ }
+
+ /**
+ * Internal method to compute hash components from any HashKey.
+ */
+ private RibbonHashResult computeHash(HashKey hashKey) {
+ // Generate two 32-bit hashes and combine into 64-bit
+ int hash1 = hash.hash(hashKey, 0);
+ int hash2 = hash.hash(hashKey, hash1);
+ long rawHash = ((long) hash1 << 32) | (hash2 & 0xFFFFFFFFL);
+
+ // Mix the hash for better distribution
+ long mixedHash = mixHash(rawHash);
+
+ // Compute start position using FastRange
+ int numStarts = numSlots - bandwidth + 1;
+ if (numStarts <= 0) {
+ numStarts = 1;
+ }
+ int start = fastRange(mixedHash >>> 32, numStarts);
+
+ // Compute coefficient row (first bit always 1)
+ long coeffRow = computeCoeffRow(mixedHash);
+
+ // For Ribbon Filter, resultRow is always 0
+ // The filter works by checking if XOR of solution values equals 0
+ // Empty rows are filled with pseudorandom data during back-substitution
+ return new RibbonHashResult(start, coeffRow, 0);
+ }
+
+ /**
+ * Mixes the hash value for better distribution using a variant of SplitMix64.
+ */
+ private long mixHash(long h) {
+ h ^= h >>> 33;
+ h *= 0xFF51AFD7ED558CCDL;
+ h ^= h >>> 33;
+ h *= 0xC4CEB9FE1A85EC53L;
+ h ^= h >>> 33;
+ return h;
+ }
+
+ /**
+ * FastRange: Maps a 32-bit hash to [0, range) using multiplication instead of modulo.
+ *
+ * This is equivalent to {@code hash % range} but much faster because multiplication requires
+ * significantly fewer CPU cycles than division.
+ *
+ * The algorithm: {@code (hash * range) >> 32} produces a uniform distribution in [0, range).
+ * @param hash 32-bit hash value (passed as long to avoid sign issues)
+ * @param range The upper bound (exclusive)
+ * @return A value in [0, range)
+ */
+ private static int fastRange(long hash, int range) {
+ // Ensure we use unsigned 32-bit multiplication
+ return (int) (((hash & 0xFFFFFFFFL) * range) >>> 32);
+ }
+
+ /**
+ * Computes the coefficient row from the mixed hash. Applies additional mixing to decorrelate from
+ * start position. The first bit is always 1 (kFirstCoeffAlwaysOne optimization).
+ */
+ private long computeCoeffRow(long mixedHash) {
+ // Apply additional mixing to decorrelate from start position
+ // Uses a different mixing constant than the main hash mixing
+ long c = mixedHash * GOLDEN_RATIO;
+ c ^= c >>> 33;
+ c *= 0xC4CEB9FE1A85EC53L;
+ c ^= c >>> 33;
+
+ // First bit is always 1 (kFirstCoeffAlwaysOne optimization)
+ // This ensures coeffRow is never zero and simplifies banding
+ return c | 1L;
+ }
+
+ /**
+ * Returns the number of slots in the ribbon filter.
+ */
+ public int getNumSlots() {
+ return numSlots;
+ }
+
+ /**
+ * Returns the bandwidth (coefficient width in bits).
+ */
+ public int getBandwidth() {
+ return bandwidth;
+ }
+
+ /**
+ * Returns the hash type.
+ */
+ public int getHashType() {
+ return hashType;
+ }
+
+ /**
+ * Result of hashing a key for Ribbon Filter.
+ * @param start Starting row position in the band matrix
+ * @param coeffRow Coefficient row (bandwidth bits wide)
+ * @param resultRow Result row (always 0)
+ */
+ public record RibbonHashResult(int start, long coeffRow, int resultRow) {
+
+ @Override
+ public String toString() {
+ return String.format("RibbonHashResult{start=%d, coeffRow=0x%016X, resultRow=0x%08X}", start,
+ coeffRow, resultRow);
+ }
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/package-info.java
new file mode 100644
index 000000000000..723b835065b8
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ribbon/package-info.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+/**
+ * Ribbon Filter implementation for HBase.
+ *
+ * Ribbon Filter is a space-efficient alternative to Bloom filters, based on the paper:
+ * "Ribbon filter: practically smaller than Bloom and Xor" by Peter C. Dillinger and
+ * Stefan Walzer (2021).
+ *
+ * Key features:
+ *
+ * - Space Efficiency: ~30% smaller than Bloom for the same FPR
+ * - Fast Queries: O(w) XOR operations where w is bandwidth (typically 64)
+ * - Batch Construction: Requires all keys before building (vs Bloom's streaming)
+ * - ICML Storage: Interleaved Column-Major Layout for space-optimal storage
+ *
+ *
+ * This implementation uses ICML (Interleaved Column-Major Layout) storage, which:
+ *
+ * - Never fails construction
+ * - Supports fractional bits per key for optimal space usage
+ *
+ *
+ * Main classes:
+ *
+ * - {@link org.apache.hadoop.hbase.util.ribbon.RibbonFilterChunk} - Main filter
+ * - {@link org.apache.hadoop.hbase.util.ribbon.RibbonHasher} - Hash generation for keys
+ * - {@link org.apache.hadoop.hbase.util.ribbon.RibbonBanding} - Gaussian elimination
+ * - {@link org.apache.hadoop.hbase.util.ribbon.InterleavedRibbonSolution} - ICML solution
+ * - {@link org.apache.hadoop.hbase.util.ribbon.RibbonFilterUtil} - Utility methods
+ *
+ *
+ * @see Ribbon Filter Paper
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.hbase.util.ribbon;
+
+import org.apache.yetus.audience.InterfaceAudience;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundRibbonFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundRibbonFilter.java
new file mode 100644
index 000000000000..186c48f85daf
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundRibbonFilter.java
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CellComparatorImpl;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.hfile.BlockCache;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.CompoundRibbonFilter;
+import org.apache.hadoop.hbase.io.hfile.CompoundRibbonFilterWriter;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileContext;
+import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
+import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.BloomFilterFactory;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests writing and reading Compound Ribbon filters in HFiles.
+ */
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestCompoundRibbonFilter {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestCompoundRibbonFilter.class);
+
+ private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestCompoundRibbonFilter.class);
+
+ // Test configurations
+ private static final int NUM_TESTS = 4;
+ private static final BloomType[] RIBBON_TYPES =
+ { BloomType.ROW, BloomType.ROW, BloomType.ROWCOL, BloomType.ROWCOL };
+
+ private static final int[] NUM_KV;
+ static {
+ final int N = 5000;
+ NUM_KV = new int[] { N, N * 2, N, N * 2 };
+ }
+
+ private static final int[] BLOCK_SIZES;
+ static {
+ final int blkSize = 65536;
+ BLOCK_SIZES = new int[] { blkSize, blkSize, blkSize, blkSize };
+ }
+
+ // Ribbon filter block sizes for testing (controls chunk size)
+ // These are chosen to create appropriate number of keys per chunk
+ // ICML uses ~7 bits/key for 1% FPR, plus overhead for slots
+ private static final int[] RIBBON_BLOCK_SIZES = { 1024, 2048, 1024, 2048 };
+
+ private static Configuration conf;
+ private static CacheConfig cacheConf;
+ private FileSystem fs;
+
+ private String testIdMsg;
+
+ private static final int GENERATION_SEED = 2319;
+ private static final int EVALUATION_SEED = 135;
+
+ private BlockCache blockCache;
+
+ @Before
+ public void setUp() throws IOException {
+ conf = TEST_UTIL.getConfiguration();
+ conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MAX_FORMAT_VERSION);
+ fs = FileSystem.get(conf);
+ blockCache = BlockCacheFactory.createBlockCache(conf);
+ cacheConf = new CacheConfig(conf, blockCache);
+ }
+
+ private List createSortedKeyValues(Random rand, int n) {
+ List kvList = new ArrayList<>(n);
+ for (int i = 0; i < n; ++i) {
+ kvList.add(RandomKeyValueUtil.randomKeyValue(rand));
+ }
+ kvList.sort(CellComparatorImpl.COMPARATOR);
+ return kvList;
+ }
+
+ @Test
+ public void testCompoundRibbonFilter() throws IOException {
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
+ for (int t = 0; t < NUM_TESTS; ++t) {
+ // Configure Ribbon filter parameters (block size controls chunk size)
+ conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, RIBBON_BLOCK_SIZES[t]);
+
+ testIdMsg = "in test #" + t + ":";
+ Random generationRand = new Random(GENERATION_SEED);
+ List kvs = createSortedKeyValues(generationRand, NUM_KV[t]);
+ BloomType bt = RIBBON_TYPES[t];
+ Path sfPath = writeStoreFile(t, bt, kvs);
+ readStoreFile(t, bt, kvs, sfPath);
+ }
+ }
+
+ private void readStoreFile(int t, BloomType bt, List kvs, Path sfPath)
+ throws IOException {
+ StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, sfPath, true);
+ HStoreFile sf = new HStoreFile(storeFileInfo, bt, cacheConf);
+ sf.initReader();
+ StoreFileReader r = sf.getReader();
+ StoreFileScanner scanner = r.getStoreFileScanner(true, true, false, 0, 0, false);
+
+ // Test for false negatives (not allowed)
+ int numChecked = 0;
+ for (KeyValue kv : kvs) {
+ byte[] row = CellUtil.cloneRow(kv);
+ boolean present = isInBloom(scanner, row, CellUtil.cloneQualifier(kv));
+ assertTrue(testIdMsg + " Ribbon filter false negative on row " + Bytes.toStringBinary(row)
+ + " after " + numChecked + " successful checks", present);
+ ++numChecked;
+ }
+
+ // Test for false positives
+ CompoundRibbonFilter crf = (CompoundRibbonFilter) r.getGeneralBloomFilter();
+ int numFalsePos = 0;
+ Random rand = new Random(EVALUATION_SEED);
+ int nTrials = NUM_KV[t] * 5;
+ for (int i = 0; i < nTrials; ++i) {
+ byte[] query = RandomKeyValueUtil.randomRowOrQualifier(rand);
+ if (isInBloom(scanner, query, rand)) {
+ numFalsePos += 1;
+ }
+ }
+ double falsePosRate = numFalsePos * 1.0 / nTrials;
+ LOG.info("{} False positives: {} out of {} ({}) - overhead={}, chunks={}", testIdMsg,
+ numFalsePos, nTrials, falsePosRate, crf.getOverheadRatio(), crf.getNumChunks());
+
+ // FPR should be within 2x of expected error rate
+ float expectedFpr = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0.01f);
+ double maxAllowedFpr = expectedFpr * 2;
+ assertTrue(
+ "False positive rate too high: " + falsePosRate + " (expected < " + maxAllowedFpr + ")",
+ falsePosRate < maxAllowedFpr);
+
+ r.close(true);
+ }
+
+ private boolean isInBloom(StoreFileScanner scanner, byte[] row, Random rand) {
+ return isInBloom(scanner, row, RandomKeyValueUtil.randomRowOrQualifier(rand));
+ }
+
+ private boolean isInBloom(StoreFileScanner scanner, byte[] row, byte[] qualifier) {
+ Scan scan = new Scan().withStartRow(row).withStopRow(row, true);
+ scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier);
+ HStore store = mock(HStore.class);
+ when(store.getColumnFamilyDescriptor())
+ .thenReturn(ColumnFamilyDescriptorBuilder.of(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
+ return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
+ }
+
+ private Path writeStoreFile(int t, BloomType bt, List kvs) throws IOException {
+ conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
+ cacheConf = new CacheConfig(conf, blockCache);
+ HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCK_SIZES[t]).build();
+ StoreFileWriter w = new StoreFileWriter.Builder(conf, cacheConf, fs)
+ .withOutputDir(TEST_UTIL.getDataTestDir()).withBloomType(bt)
+ .withBloomFilterImpl(BloomFilterImpl.RIBBON).withFileContext(meta).build();
+
+ assertTrue(w.hasGeneralBloom());
+ assertTrue(
+ "Expected CompoundRibbonFilterWriter but got " + w.getGeneralBloomWriter().getClass(),
+ w.getGeneralBloomWriter() instanceof CompoundRibbonFilterWriter);
+ CompoundRibbonFilterWriter crfw = (CompoundRibbonFilterWriter) w.getGeneralBloomWriter();
+
+ int keyCount = 0;
+ KeyValue prev = null;
+ LOG.debug("Total keys/values to insert: {}", kvs.size());
+ for (KeyValue kv : kvs) {
+ w.append(kv);
+
+ // Validate the key count
+ boolean newKey = true;
+ if (prev != null) {
+ newKey = !(bt == BloomType.ROW
+ ? CellUtil.matchingRows(kv, prev)
+ : CellUtil.matchingRowColumn(kv, prev));
+ }
+ if (newKey) {
+ ++keyCount;
+ }
+ assertEquals(keyCount, crfw.getKeyCount());
+
+ prev = kv;
+ }
+ w.close();
+
+ return w.getPath();
+ }
+
+ @Test
+ public void testPerChunkNumSlotsMetadata() throws IOException {
+ // Test that per-chunk numSlots are correctly stored in metadata
+ // by creating multiple chunks with different sizes
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
+ // Small block size to create multiple chunks (~500 keys per chunk)
+ conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, 550);
+
+ testIdMsg = "in per-chunk numSlots test:";
+ Random generationRand = new Random(GENERATION_SEED);
+ // 2500 keys with ~500 max per chunk = ~5 chunks
+ List kvs = createSortedKeyValues(generationRand, 2500);
+
+ Path sfPath = writeStoreFile(0, BloomType.ROW, kvs);
+
+ // Read and verify per-chunk numSlots metadata
+ StoreFileInfo storeFileInfo = StoreFileInfo.createStoreFileInfoForHFile(conf, fs, sfPath, true);
+ HStoreFile sf = new HStoreFile(storeFileInfo, BloomType.ROW, cacheConf);
+ sf.initReader();
+ StoreFileReader r = sf.getReader();
+ CompoundRibbonFilter crf = (CompoundRibbonFilter) r.getGeneralBloomFilter();
+
+ int numChunks = crf.getNumChunks();
+ LOG.info("{} Created {} chunks", testIdMsg, numChunks);
+ assertTrue("Should have multiple chunks", numChunks > 1);
+
+ // Verify all keys are found (this confirms metadata is read correctly)
+ StoreFileScanner scanner = r.getStoreFileScanner(true, true, false, 0, 0, false);
+ int numChecked = 0;
+ for (KeyValue kv : kvs) {
+ byte[] row = CellUtil.cloneRow(kv);
+ boolean present = isInBloom(scanner, row, CellUtil.cloneQualifier(kv));
+ assertTrue(testIdMsg + " Ribbon filter false negative on row " + Bytes.toStringBinary(row),
+ present);
+ ++numChecked;
+ }
+ LOG.info("{} Verified {} keys across {} chunks with per-chunk numSlots metadata", testIdMsg,
+ numChecked, numChunks);
+
+ r.close(true);
+ }
+
+ @Test
+ public void testDeleteFamilyRibbonFilter() throws IOException {
+ // Enable Delete Family Bloom filter (Ribbon type will be used because general bloom is Ribbon)
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
+ conf.setBoolean(BloomFilterFactory.IO_STOREFILE_DELETEFAMILY_BLOOM_ENABLED, true);
+
+ testIdMsg = "in Delete Family Ribbon test:";
+
+ // Write the file
+ Path testDir = TEST_UTIL.getDataTestDir("delete_family_ribbon_test");
+ fs.mkdirs(testDir);
+ Path filePath = StoreFileWriter.getUniqueFile(fs, testDir);
+
+ HFileContext meta = new HFileContextBuilder().withBlockSize(65536).build();
+ // Use ROW + RIBBON impl so delete family filter also uses Ribbon
+ StoreFileWriter writer = new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(filePath)
+ .withBloomType(BloomType.ROW).withBloomFilterImpl(BloomFilterImpl.RIBBON)
+ .withMaxKeyCount(2000).withFileContext(meta).build();
+
+ // Add delete family markers for even rows
+ long now = System.currentTimeMillis();
+ for (int i = 0; i < 2000; i += 2) {
+ String row = String.format("%08d", i);
+ KeyValue kv = new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("col"),
+ now, KeyValue.Type.DeleteFamily, Bytes.toBytes("value"));
+ writer.append(kv);
+ }
+ writer.close();
+
+ // Read and verify
+ StoreFileInfo storeFileInfo =
+ StoreFileInfo.createStoreFileInfoForHFile(conf, fs, filePath, true);
+ HStoreFile sf = new HStoreFile(storeFileInfo, BloomType.ROW, cacheConf);
+ sf.initReader();
+ StoreFileReader reader = sf.getReader();
+
+ // Check that delete family ribbon filter works
+ int falsePos = 0;
+ int falseNeg = 0;
+ for (int i = 0; i < 2000; i++) {
+ String row = String.format("%08d", i);
+ byte[] rowKey = Bytes.toBytes(row);
+ boolean exists = reader.passesDeleteFamilyBloomFilter(rowKey, 0, rowKey.length);
+ if (i % 2 == 0) {
+ // Even rows have delete family markers - should be found
+ if (!exists) {
+ falseNeg++;
+ }
+ } else {
+ // Odd rows don't have delete family markers - false positives counted
+ if (exists) {
+ falsePos++;
+ }
+ }
+ }
+
+ // There should be 1000 delete family markers
+ assertEquals(testIdMsg + " Delete family count should be 1000", 1000,
+ reader.getDeleteFamilyCnt());
+
+ // No false negatives allowed
+ assertEquals(testIdMsg + " Should have no false negatives", 0, falseNeg);
+
+ // FPR should be within 2x of expected error rate
+ double fpr = (double) falsePos / 1000;
+ LOG.info("{} Delete Family Ribbon - False positives: {} out of 1000 (FPR={})", testIdMsg,
+ falsePos, fpr);
+ float expectedFpr = conf.getFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0.01f);
+ double maxAllowedFpr = expectedFpr * 2;
+ assertTrue("False positive rate too high: " + fpr + " (expected < " + maxAllowedFpr + ")",
+ fpr < maxAllowedFpr);
+
+ reader.close(true);
+
+ LOG.info("{} Successfully verified Delete Family Ribbon filter", testIdMsg);
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestInterleavedRibbonSolution.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestInterleavedRibbonSolution.java
new file mode 100644
index 000000000000..96579c8df157
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestInterleavedRibbonSolution.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.nio.SingleByteBuff;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Hash;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for {@link InterleavedRibbonSolution}.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestInterleavedRibbonSolution {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestInterleavedRibbonSolution.class);
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestInterleavedRibbonSolution.class);
+
+ private static final int COEFF_BITS = InterleavedRibbonSolution.COEFF_BITS;
+
+ @Test
+ public void testBandingAndBackSubstitution() {
+ int numSlots = InterleavedRibbonSolution.roundUpNumSlots(1000);
+ int numKeys = 800;
+
+ RibbonBanding banding = new RibbonBanding(numSlots, COEFF_BITS);
+ RibbonHasher hasher = new RibbonHasher(numSlots, COEFF_BITS, Hash.MURMUR_HASH3);
+
+ RibbonHasher.RibbonHashResult[] results = new RibbonHasher.RibbonHashResult[numKeys];
+ for (int i = 0; i < numKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ results[i] = hasher.hash(key, 0, key.length);
+ banding.add(results[i].start(), results[i].coeffRow());
+ }
+
+ assertEquals(numKeys, banding.getNumAdded());
+
+ InterleavedRibbonSolution solution = new InterleavedRibbonSolution(numSlots, 0.01);
+ solution.backSubstFrom(banding);
+
+ // All added keys must be found (no false negatives)
+ for (int i = 0; i < numKeys; i++) {
+ assertTrue("Key " + i + " must be found",
+ solution.contains(results[i].start(), results[i].coeffRow(), results[i].resultRow()));
+ }
+ }
+
+ @Test
+ public void testContains() {
+ int numSlots = InterleavedRibbonSolution.roundUpNumSlots(500);
+ int numKeys = 400;
+
+ RibbonBanding banding = new RibbonBanding(numSlots, COEFF_BITS);
+ RibbonHasher hasher = new RibbonHasher(numSlots, COEFF_BITS, Hash.MURMUR_HASH3);
+
+ RibbonHasher.RibbonHashResult[] results = new RibbonHasher.RibbonHashResult[numKeys];
+ for (int i = 0; i < numKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ results[i] = hasher.hash(key, 0, key.length);
+ banding.add(results[i].start(), results[i].coeffRow());
+ }
+
+ InterleavedRibbonSolution solution = new InterleavedRibbonSolution(numSlots, 0.01);
+ solution.backSubstFrom(banding);
+
+ // Verify static contains method with ByteBuff (used for reading from HFile)
+ long[] segments = solution.getSegments();
+ ByteBuffer bb = ByteBuffer.allocate(segments.length * Long.BYTES);
+ bb.asLongBuffer().put(segments);
+ SingleByteBuff buf = new SingleByteBuff(bb);
+
+ for (int i = 0; i < numKeys; i++) {
+ assertTrue("Key " + i + " must be found via static contains",
+ InterleavedRibbonSolution.contains(results[i].start(), results[i].coeffRow(),
+ results[i].resultRow(), buf, 0, numSlots, solution.getUpperNumColumns(),
+ solution.getUpperStartBlock()));
+ }
+ }
+
+ @Test
+ public void testFalsePositiveRate() {
+ int numSlots = InterleavedRibbonSolution.roundUpNumSlots(5000);
+ int numKeys = 4000;
+ int numTests = 50000;
+ double targetFpRate = 0.01;
+
+ RibbonBanding banding = new RibbonBanding(numSlots, COEFF_BITS);
+ RibbonHasher hasher = new RibbonHasher(numSlots, COEFF_BITS, Hash.MURMUR_HASH3);
+
+ for (int i = 0; i < numKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ RibbonHasher.RibbonHashResult result = hasher.hash(key, 0, key.length);
+ banding.add(result.start(), result.coeffRow());
+ }
+
+ InterleavedRibbonSolution solution = new InterleavedRibbonSolution(numSlots, targetFpRate);
+ solution.backSubstFrom(banding);
+
+ // Test with keys that were NOT added (use range numKeys to numKeys + numTests)
+ int falsePositives = 0;
+ for (int i = 0; i < numTests; i++) {
+ byte[] testKey = Bytes.toBytes(numKeys + i);
+ RibbonHasher.RibbonHashResult result = hasher.hash(testKey, 0, testKey.length);
+ if (solution.contains(result.start(), result.coeffRow(), result.resultRow())) {
+ falsePositives++;
+ }
+ }
+
+ double actualFpr = (double) falsePositives / numTests;
+ LOG.info("False positives: {} / {}", falsePositives, numTests);
+ LOG.info("Actual FPR: {}", String.format("%.4f%%", actualFpr * 100));
+
+ // FPR should be within reasonable bounds (allow 5x margin for statistical variance)
+ assertTrue("FPR should be less than 5% for 1% target", actualFpr < 0.05);
+ }
+
+ @Test
+ public void testColumnLayout() {
+ int numSlots = 1024;
+
+ // Different FP rates should result in different column layouts
+ InterleavedRibbonSolution sol1 = new InterleavedRibbonSolution(numSlots, 0.01); // 1%
+ InterleavedRibbonSolution sol2 = new InterleavedRibbonSolution(numSlots, 0.001); // 0.1%
+
+ LOG.info("1% FPR: columns={}, byteSize={}", sol1.getUpperNumColumns(), sol1.getByteSize());
+ LOG.info("0.1% FPR: columns={}, byteSize={}", sol2.getUpperNumColumns(), sol2.getByteSize());
+
+ // Lower FPR needs more columns (bits per key)
+ assertTrue(sol2.getUpperNumColumns() > sol1.getUpperNumColumns());
+ assertTrue(sol2.getByteSize() > sol1.getByteSize());
+ }
+
+ @Test
+ public void testRoundUpNumSlots() {
+ // Minimum is 2 * COEFF_BITS = 128
+ assertEquals(128, InterleavedRibbonSolution.roundUpNumSlots(1));
+ assertEquals(128, InterleavedRibbonSolution.roundUpNumSlots(128));
+ assertEquals(192, InterleavedRibbonSolution.roundUpNumSlots(129));
+ assertEquals(1024, InterleavedRibbonSolution.roundUpNumSlots(1000));
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonBanding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonBanding.java
new file mode 100644
index 000000000000..74c6908717df
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonBanding.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests for {@link RibbonBanding}.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestRibbonBanding {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestRibbonBanding.class);
+
+ private static final int COEFF_BITS = InterleavedRibbonSolution.COEFF_BITS;
+
+ @Test
+ public void testGaussianEliminationXOR() {
+ // Test Gaussian elimination with XOR cascade
+ RibbonBanding banding = new RibbonBanding(1000, COEFF_BITS);
+
+ // Add entries that will trigger XOR cascade
+ banding.add(0, 0b1111L); // 1111 at position 0
+ banding.add(0, 0b1011L); // XOR with first -> 0100, shifts to position 2
+ banding.add(0, 0b1101L); // XOR with first -> 0010, shifts to position 1
+
+ assertEquals(3, banding.getNumAdded());
+ assertEquals(0b1111L, banding.getCoeffRow(0));
+ assertEquals(0b0001L, banding.getCoeffRow(1)); // 0010 shifted by 1
+ assertEquals(0b0001L, banding.getCoeffRow(2)); // 0100 shifted by 2
+
+ // Test redundant equation (XOR results in 0)
+ banding.add(0, 0b1111L); // Same as first, XOR gives 0, absorbed
+ assertEquals(3, banding.getNumAdded()); // Still 3
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterChunk.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterChunk.java
new file mode 100644
index 000000000000..f9d4c85429ba
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterChunk.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for {@link RibbonFilterChunk}.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestRibbonFilterChunk {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestRibbonFilterChunk.class);
+
+ private static final Logger LOG = LoggerFactory.getLogger(TestRibbonFilterChunk.class);
+
+ @Test
+ public void testBasicRibbon() {
+ RibbonFilterChunk rf1 = new RibbonFilterChunk(BloomType.ROW);
+ RibbonFilterChunk rf2 = new RibbonFilterChunk(BloomType.ROW);
+ rf1.allocRibbon(1000);
+ rf2.allocRibbon(1000);
+
+ byte[] key1 = Bytes.toBytes(1);
+ byte[] key2 = Bytes.toBytes(2);
+
+ rf1.addKey(key1);
+ rf2.addKey(key2);
+
+ rf1.finalizeRibbon();
+ rf2.finalizeRibbon();
+
+ // No false negatives
+ assertTrue(rf1.contains(key1, 0, key1.length));
+ assertTrue(rf2.contains(key2, 0, key2.length));
+ }
+
+ @Test
+ public void testSerialization() throws Exception {
+ RibbonFilterChunk rf = new RibbonFilterChunk(BloomType.ROW);
+ rf.allocRibbon(1000);
+
+ for (int i = 0; i < 100; i++) {
+ rf.addKey(Bytes.toBytes(i));
+ }
+ rf.finalizeRibbon();
+
+ ByteArrayOutputStream bOut = new ByteArrayOutputStream();
+ rf.writeRibbon(new DataOutputStream(bOut));
+
+ assertTrue(bOut.size() > 0);
+ LOG.info("Serialized {} keys as {} bytes", rf.getKeyCount(), bOut.size());
+ }
+
+ @Test
+ public void testFalsePositiveRate() {
+ int numKeys = 10000;
+ int numTests = 100000;
+
+ RibbonFilterChunk rf = new RibbonFilterChunk(BloomType.ROW);
+ rf.allocRibbon(numKeys);
+
+ for (int i = 0; i < numKeys; i++) {
+ rf.addKey(Bytes.toBytes(i));
+ }
+ rf.finalizeRibbon();
+
+ // Verify no false negatives
+ for (int i = 0; i < numKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ assertTrue("Key " + i + " should be found", rf.contains(key, 0, key.length));
+ }
+
+ // Measure false positive rate with non-added keys
+ int falsePositives = 0;
+ for (int i = 0; i < numTests; i++) {
+ byte[] key = Bytes.toBytes(numKeys + i);
+ if (rf.contains(key, 0, key.length)) {
+ falsePositives++;
+ }
+ }
+
+ double actualFpr = (double) falsePositives / numTests;
+ LOG.info("FPR: {} / {} = {}%", falsePositives, numTests,
+ String.format("%.2f", actualFpr * 100));
+ assertTrue("FPR should be less than 5% for 1% target", actualFpr < 0.05);
+ }
+
+ @Test
+ public void testLazyAllocation() {
+ // Ribbon Filter uses lazy allocation - filter is sized based on actual keys added,
+ // not the maxKeys passed to allocRibbon. This is for the space efficiency.
+ int maxKeys = 10000;
+ int actualKeys = 100;
+
+ RibbonFilterChunk rf = new RibbonFilterChunk(BloomType.ROW);
+ rf.allocRibbon(maxKeys);
+
+ for (int i = 0; i < actualKeys; i++) {
+ rf.addKey(Bytes.toBytes(i));
+ }
+ rf.finalizeRibbon();
+
+ // Verify all keys are found
+ for (int i = 0; i < actualKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ assertTrue("Should find key " + i, rf.contains(key, 0, key.length));
+ }
+
+ // numSlots should be sized for actualKeys, not maxKeys
+ // Use the same overhead ratio that the default constructor uses (for 1% FPR)
+ double defaultOverhead =
+ RibbonFilterUtil.computeOptimalOverheadForFpRate(0.01, RibbonFilterUtil.DEFAULT_BANDWIDTH);
+ int slotsForMaxKeys = InterleavedRibbonSolution
+ .roundUpNumSlots(RibbonFilterUtil.computeNumSlots(maxKeys, defaultOverhead));
+ assertTrue("Filter should NOT be sized for maxKeys", rf.getNumSlots() < slotsForMaxKeys);
+
+ LOG.info("maxKeys={}, actualKeys={}, slotsForMaxKeys={}, actualSlots={}", maxKeys, actualKeys,
+ slotsForMaxKeys, rf.getNumSlots());
+ }
+
+ @Test
+ public void testDifferentFpRates() {
+ int numKeys = 10000;
+
+ // Test with 1% FP rate (default)
+ RibbonFilterChunk rf1 = new RibbonFilterChunk(BloomType.ROW);
+ rf1.allocRibbon(numKeys);
+ for (int i = 0; i < numKeys; i++) {
+ rf1.addKey(Bytes.toBytes(i));
+ }
+ rf1.finalizeRibbon();
+
+ // Test with 0.1% FP rate
+ double fpRate2 = 0.001;
+ double overhead2 =
+ RibbonFilterUtil.computeOptimalOverheadForFpRate(fpRate2, RibbonFilterUtil.DEFAULT_BANDWIDTH);
+ RibbonFilterChunk rf2 = new RibbonFilterChunk(RibbonFilterUtil.DEFAULT_BANDWIDTH,
+ RibbonFilterUtil.getDefaultHashType(), BloomType.ROW, fpRate2, overhead2);
+ rf2.allocRibbon(numKeys);
+ for (int i = 0; i < numKeys; i++) {
+ rf2.addKey(Bytes.toBytes(i));
+ }
+ rf2.finalizeRibbon();
+
+ LOG.info("1% FPR: byteSize={}, bitsPerKey={}", rf1.getByteSize(),
+ String.format("%.2f", rf1.getByteSize() * 8.0 / numKeys));
+ LOG.info("0.1% FPR: byteSize={}, bitsPerKey={}", rf2.getByteSize(),
+ String.format("%.2f", rf2.getByteSize() * 8.0 / numKeys));
+
+ // Lower FP rate should use more space
+ assertTrue("Lower FP rate should use more bits per key", rf2.getByteSize() > rf1.getByteSize());
+
+ // Verify all keys are found
+ for (int i = 0; i < numKeys; i++) {
+ byte[] key = Bytes.toBytes(i);
+ assertTrue(rf1.contains(key, 0, key.length));
+ assertTrue(rf2.contains(key, 0, key.length));
+ }
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterUtil.java
new file mode 100644
index 000000000000..7222c016daa4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonFilterUtil.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests for {@link RibbonFilterUtil}.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestRibbonFilterUtil {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestRibbonFilterUtil.class);
+
+ @Test
+ public void testComputeNumSlots() {
+ // Basic computation: keys * (1 + overhead)
+ assertEquals(1050, RibbonFilterUtil.computeNumSlots(1000, 0.05));
+ assertEquals(1100, RibbonFilterUtil.computeNumSlots(1000, 0.10));
+
+ // Zero keys returns 0
+ assertEquals(0, RibbonFilterUtil.computeNumSlots(0, 0.05));
+
+ // Small number of keys should still get minimum slots (bandwidth)
+ int minSlots = RibbonFilterUtil.computeNumSlots(10, 0.05);
+ assertTrue("Should have at least bandwidth slots",
+ minSlots >= RibbonFilterUtil.DEFAULT_BANDWIDTH);
+
+ // Overhead clamping
+ int slots1 = RibbonFilterUtil.computeNumSlots(1000, 0.001); // below min
+ int slots2 = RibbonFilterUtil.computeNumSlots(1000, RibbonFilterUtil.MIN_OVERHEAD_RATIO);
+ assertEquals(slots1, slots2);
+ }
+
+ @Test
+ public void testComputeOptimalOverhead() {
+ // Based on paper's Equation 7: ε = (4 + r/4) / w where r = -log₂(fpRate)
+ // Hand-calculated expected values:
+ // - 1% FPR: r = 6.644, ε = (4 + 1.661) / 64 = 0.0885
+ // - 0.1% FPR: r = 9.966, ε = (4 + 2.491) / 64 = 0.1014
+ int bandwidth = RibbonFilterUtil.DEFAULT_BANDWIDTH;
+
+ double overhead1 = RibbonFilterUtil.computeOptimalOverheadForFpRate(0.01, bandwidth);
+ double overhead2 = RibbonFilterUtil.computeOptimalOverheadForFpRate(0.001, bandwidth);
+
+ assertEquals("Overhead for 1% FPR", 0.0885, overhead1, 0.001);
+ assertEquals("Overhead for 0.1% FPR", 0.1014, overhead2, 0.001);
+
+ // Lower FPR requires higher overhead
+ assertTrue("Lower FPR requires higher overhead", overhead2 > overhead1);
+ }
+
+ @Test
+ public void testComputeFingerprintBits() {
+ // ceil(-log₂(fpRate))
+ assertEquals(7, RibbonFilterUtil.computeFingerprintBits(0.01)); // 1% FPR
+ assertEquals(10, RibbonFilterUtil.computeFingerprintBits(0.001)); // 0.1% FPR
+ assertEquals(1, RibbonFilterUtil.computeFingerprintBits(0.5)); // 50% FPR
+ }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonHasher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonHasher.java
new file mode 100644
index 000000000000..b08d5df86940
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/ribbon/TestRibbonHasher.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util.ribbon;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Hash;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Tests for {@link RibbonHasher}.
+ */
+@Category({ MiscTests.class, SmallTests.class })
+public class TestRibbonHasher {
+
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestRibbonHasher.class);
+
+ @Test
+ public void testBasicHashing() {
+ int numSlots = 1000;
+ int bandwidth = RibbonFilterUtil.DEFAULT_BANDWIDTH;
+ RibbonHasher hasher = new RibbonHasher(numSlots, bandwidth, Hash.MURMUR_HASH3);
+
+ // Test hash consistency: same key produces same result
+ byte[] key = Bytes.toBytes(0);
+ RibbonHasher.RibbonHashResult r1 = hasher.hash(key, 0, key.length);
+ RibbonHasher.RibbonHashResult r2 = hasher.hash(key, 0, key.length);
+ assertEquals(r1.start(), r2.start());
+ assertEquals(r1.coeffRow(), r2.coeffRow());
+ assertEquals(r1.resultRow(), r2.resultRow());
+
+ // Test hash properties for many keys
+ int maxStart = numSlots - bandwidth + 1;
+ for (int i = 0; i < 1000; i++) {
+ byte[] k = Bytes.toBytes(i);
+ RibbonHasher.RibbonHashResult result = hasher.hash(k, 0, k.length);
+
+ // start must be in valid range [0, numSlots - bandwidth]
+ assertTrue("Start should be >= 0", result.start() >= 0);
+ assertTrue("Start should be < " + maxStart, result.start() < maxStart);
+
+ // First coefficient bit is always 1 (kFirstCoeffAlwaysOne optimization)
+ assertEquals("First coeff bit should be 1", 1, result.coeffRow() & 1);
+
+ // For Homogeneous Ribbon Filter, resultRow is always 0
+ assertEquals("Result row should be 0", 0, result.resultRow());
+ }
+ }
+}
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
index 23caaf26e00e..3eb3653ac2f9 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
@@ -71,12 +71,14 @@
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.regionserver.BloomFilterImpl;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.thrift2.generated.TAppend;
import org.apache.hadoop.hbase.thrift2.generated.TAuthorization;
+import org.apache.hadoop.hbase.thrift2.generated.TBloomFilterImpl;
import org.apache.hadoop.hbase.thrift2.generated.TBloomFilterType;
import org.apache.hadoop.hbase.thrift2.generated.TCellVisibility;
import org.apache.hadoop.hbase.thrift2.generated.TColumn;
@@ -903,6 +905,16 @@ public static BloomType bloomFilterFromThrift(TBloomFilterType in) {
}
}
+ public static BloomFilterImpl bloomFilterImplFromThrift(TBloomFilterImpl in) {
+ switch (in.getValue()) {
+ case 1:
+ return BloomFilterImpl.RIBBON;
+ case 0:
+ default:
+ return BloomFilterImpl.BLOOM;
+ }
+ }
+
public static Compression.Algorithm compressionAlgorithmFromThrift(TCompressionAlgorithm in) {
switch (in.getValue()) {
case 0:
@@ -974,6 +986,9 @@ public static KeepDeletedCells keepDeletedCellsFromThrift(TKeepDeletedCells in)
if (in.isSetBloomnFilterType()) {
builder.setBloomFilterType(bloomFilterFromThrift(in.getBloomnFilterType()));
}
+ if (in.isSetBloomFilterImpl()) {
+ builder.setBloomFilterImpl(bloomFilterImplFromThrift(in.getBloomFilterImpl()));
+ }
if (in.isSetCompressionType()) {
builder.setCompressionType(compressionAlgorithmFromThrift(in.getCompressionType()));
}
@@ -1138,6 +1153,16 @@ public static TBloomFilterType bloomFilterFromHBase(BloomType in) {
}
}
+ public static TBloomFilterImpl bloomFilterImplFromHBase(BloomFilterImpl in) {
+ switch (in) {
+ case RIBBON:
+ return TBloomFilterImpl.RIBBON;
+ case BLOOM:
+ default:
+ return TBloomFilterImpl.BLOOM;
+ }
+ }
+
public static TCompressionAlgorithm compressionAlgorithmFromHBase(Compression.Algorithm in) {
switch (in) {
case LZO:
@@ -1201,6 +1226,7 @@ public static TColumnFamilyDescriptor columnFamilyDescriptorFromHBase(ColumnFami
}
out.setBlockSize(in.getBlocksize());
out.setBloomnFilterType(bloomFilterFromHBase(in.getBloomFilterType()));
+ out.setBloomFilterImpl(bloomFilterImplFromHBase(in.getBloomFilterImpl()));
out.setCompressionType(compressionAlgorithmFromHBase(in.getCompressionType()));
out.setDfsReplication(in.getDFSReplication());
out.setDataBlockEncoding(dataBlockEncodingFromHBase(in.getDataBlockEncoding()));
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterImpl.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterImpl.java
new file mode 100644
index 000000000000..369974e4e68a
--- /dev/null
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterImpl.java
@@ -0,0 +1,53 @@
+/**
+ * Autogenerated by Thrift Compiler (0.14.1)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ * @generated
+ */
+package org.apache.hadoop.hbase.thrift2.generated;
+
+
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.regionserver.BloomFilterImpl
+ */
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2026-01-26")
+public enum TBloomFilterImpl implements org.apache.thrift.TEnum {
+ /**
+ * Traditional Bloom filter implementation
+ */
+ BLOOM(0),
+ /**
+ * Ribbon filter implementation (more space-efficient than Bloom filters)
+ */
+ RIBBON(1);
+
+ private final int value;
+
+ private TBloomFilterImpl(int value) {
+ this.value = value;
+ }
+
+ /**
+ * Get the integer value of this enum value, as defined in the Thrift IDL.
+ */
+ public int getValue() {
+ return value;
+ }
+
+ /**
+ * Find a the enum type by its integer value, as defined in the Thrift IDL.
+ * @return null if the value is not found.
+ */
+ @org.apache.thrift.annotation.Nullable
+ public static TBloomFilterImpl findByValue(int value) {
+ switch (value) {
+ case 0:
+ return BLOOM;
+ case 1:
+ return RIBBON;
+ default:
+ return null;
+ }
+ }
+}
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
index f62eb36b0e78..ddd60fc9f1f6 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TBloomFilterType.java
@@ -11,7 +11,7 @@
* Thrift wrapper around
* org.apache.hadoop.hbase.regionserver.BloomType
*/
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2025-08-16")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2026-01-26")
public enum TBloomFilterType implements org.apache.thrift.TEnum {
/**
* Bloomfilters disabled
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
index 74e07b516935..6a79c702d066 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnFamilyDescriptor.java
@@ -11,7 +11,7 @@
* Thrift wrapper around
* org.apache.hadoop.hbase.client.ColumnFamilyDescriptor
*/
-@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2025-08-16")
+@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.14.1)", date = "2026-01-26")
public class TColumnFamilyDescriptor implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable {
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnFamilyDescriptor");
@@ -35,6 +35,7 @@ public class TColumnFamilyDescriptor implements org.apache.thrift.TBase byName = new java.util.HashMap();
@@ -169,6 +180,8 @@ public static _Fields findByThriftId(int fieldId) {
return EVICT_BLOCKS_ON_CLOSE;
case 20: // IN_MEMORY
return IN_MEMORY;
+ case 21: // BLOOM_FILTER_IMPL
+ return BLOOM_FILTER_IMPL;
default:
return null;
}
@@ -224,7 +237,7 @@ public java.lang.String getFieldName() {
private static final int __EVICTBLOCKSONCLOSE_ISSET_ID = 11;
private static final int __INMEMORY_ISSET_ID = 12;
private short __isset_bitfield = 0;
- private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.CONFIGURATION,_Fields.BLOCK_SIZE,_Fields.BLOOMN_FILTER_TYPE,_Fields.COMPRESSION_TYPE,_Fields.DFS_REPLICATION,_Fields.DATA_BLOCK_ENCODING,_Fields.KEEP_DELETED_CELLS,_Fields.MAX_VERSIONS,_Fields.MIN_VERSIONS,_Fields.SCOPE,_Fields.TIME_TO_LIVE,_Fields.BLOCK_CACHE_ENABLED,_Fields.CACHE_BLOOMS_ON_WRITE,_Fields.CACHE_DATA_ON_WRITE,_Fields.CACHE_INDEXES_ON_WRITE,_Fields.COMPRESS_TAGS,_Fields.EVICT_BLOCKS_ON_CLOSE,_Fields.IN_MEMORY};
+ private static final _Fields optionals[] = {_Fields.ATTRIBUTES,_Fields.CONFIGURATION,_Fields.BLOCK_SIZE,_Fields.BLOOMN_FILTER_TYPE,_Fields.COMPRESSION_TYPE,_Fields.DFS_REPLICATION,_Fields.DATA_BLOCK_ENCODING,_Fields.KEEP_DELETED_CELLS,_Fields.MAX_VERSIONS,_Fields.MIN_VERSIONS,_Fields.SCOPE,_Fields.TIME_TO_LIVE,_Fields.BLOCK_CACHE_ENABLED,_Fields.CACHE_BLOOMS_ON_WRITE,_Fields.CACHE_DATA_ON_WRITE,_Fields.CACHE_INDEXES_ON_WRITE,_Fields.COMPRESS_TAGS,_Fields.EVICT_BLOCKS_ON_CLOSE,_Fields.IN_MEMORY,_Fields.BLOOM_FILTER_IMPL};
public static final java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
static {
java.util.Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new java.util.EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -272,6 +285,8 @@ public java.lang.String getFieldName() {
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
tmpMap.put(_Fields.IN_MEMORY, new org.apache.thrift.meta_data.FieldMetaData("inMemory", org.apache.thrift.TFieldRequirementType.OPTIONAL,
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)));
+ tmpMap.put(_Fields.BLOOM_FILTER_IMPL, new org.apache.thrift.meta_data.FieldMetaData("bloomFilterImpl", org.apache.thrift.TFieldRequirementType.OPTIONAL,
+ new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TBloomFilterImpl.class)));
metaDataMap = java.util.Collections.unmodifiableMap(tmpMap);
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnFamilyDescriptor.class, metaDataMap);
}
@@ -327,6 +342,9 @@ public TColumnFamilyDescriptor(TColumnFamilyDescriptor other) {
this.compressTags = other.compressTags;
this.evictBlocksOnClose = other.evictBlocksOnClose;
this.inMemory = other.inMemory;
+ if (other.isSetBloomFilterImpl()) {
+ this.bloomFilterImpl = other.bloomFilterImpl;
+ }
}
public TColumnFamilyDescriptor deepCopy() {
@@ -368,6 +386,7 @@ public void clear() {
this.evictBlocksOnClose = false;
setInMemoryIsSet(false);
this.inMemory = false;
+ this.bloomFilterImpl = null;
}
public byte[] getName() {
@@ -907,6 +926,39 @@ public void setInMemoryIsSet(boolean value) {
__isset_bitfield = org.apache.thrift.EncodingUtils.setBit(__isset_bitfield, __INMEMORY_ISSET_ID, value);
}
+ /**
+ *
+ * @see TBloomFilterImpl
+ */
+ @org.apache.thrift.annotation.Nullable
+ public TBloomFilterImpl getBloomFilterImpl() {
+ return this.bloomFilterImpl;
+ }
+
+ /**
+ *
+ * @see TBloomFilterImpl
+ */
+ public TColumnFamilyDescriptor setBloomFilterImpl(@org.apache.thrift.annotation.Nullable TBloomFilterImpl bloomFilterImpl) {
+ this.bloomFilterImpl = bloomFilterImpl;
+ return this;
+ }
+
+ public void unsetBloomFilterImpl() {
+ this.bloomFilterImpl = null;
+ }
+
+ /** Returns true if field bloomFilterImpl is set (has been assigned a value) and false otherwise */
+ public boolean isSetBloomFilterImpl() {
+ return this.bloomFilterImpl != null;
+ }
+
+ public void setBloomFilterImplIsSet(boolean value) {
+ if (!value) {
+ this.bloomFilterImpl = null;
+ }
+ }
+
public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable java.lang.Object value) {
switch (field) {
case NAME:
@@ -1073,6 +1125,14 @@ public void setFieldValue(_Fields field, @org.apache.thrift.annotation.Nullable
}
break;
+ case BLOOM_FILTER_IMPL:
+ if (value == null) {
+ unsetBloomFilterImpl();
+ } else {
+ setBloomFilterImpl((TBloomFilterImpl)value);
+ }
+ break;
+
}
}
@@ -1139,6 +1199,9 @@ public java.lang.Object getFieldValue(_Fields field) {
case IN_MEMORY:
return isInMemory();
+ case BLOOM_FILTER_IMPL:
+ return getBloomFilterImpl();
+
}
throw new java.lang.IllegalStateException();
}
@@ -1190,6 +1253,8 @@ public boolean isSet(_Fields field) {
return isSetEvictBlocksOnClose();
case IN_MEMORY:
return isSetInMemory();
+ case BLOOM_FILTER_IMPL:
+ return isSetBloomFilterImpl();
}
throw new java.lang.IllegalStateException();
}
@@ -1387,6 +1452,15 @@ public boolean equals(TColumnFamilyDescriptor that) {
return false;
}
+ boolean this_present_bloomFilterImpl = true && this.isSetBloomFilterImpl();
+ boolean that_present_bloomFilterImpl = true && that.isSetBloomFilterImpl();
+ if (this_present_bloomFilterImpl || that_present_bloomFilterImpl) {
+ if (!(this_present_bloomFilterImpl && that_present_bloomFilterImpl))
+ return false;
+ if (!this.bloomFilterImpl.equals(that.bloomFilterImpl))
+ return false;
+ }
+
return true;
}
@@ -1474,6 +1548,10 @@ public int hashCode() {
if (isSetInMemory())
hashCode = hashCode * 8191 + ((inMemory) ? 131071 : 524287);
+ hashCode = hashCode * 8191 + ((isSetBloomFilterImpl()) ? 131071 : 524287);
+ if (isSetBloomFilterImpl())
+ hashCode = hashCode * 8191 + bloomFilterImpl.getValue();
+
return hashCode;
}
@@ -1685,6 +1763,16 @@ public int compareTo(TColumnFamilyDescriptor other) {
return lastComparison;
}
}
+ lastComparison = java.lang.Boolean.compare(isSetBloomFilterImpl(), other.isSetBloomFilterImpl());
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ if (isSetBloomFilterImpl()) {
+ lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.bloomFilterImpl, other.bloomFilterImpl);
+ if (lastComparison != 0) {
+ return lastComparison;
+ }
+ }
return 0;
}
@@ -1851,6 +1939,16 @@ public java.lang.String toString() {
sb.append(this.inMemory);
first = false;
}
+ if (isSetBloomFilterImpl()) {
+ if (!first) sb.append(", ");
+ sb.append("bloomFilterImpl:");
+ if (this.bloomFilterImpl == null) {
+ sb.append("null");
+ } else {
+ sb.append(this.bloomFilterImpl);
+ }
+ first = false;
+ }
sb.append(")");
return sb.toString();
}
@@ -2083,6 +2181,14 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnFamilyDescri
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
break;
+ case 21: // BLOOM_FILTER_IMPL
+ if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+ struct.bloomFilterImpl = org.apache.hadoop.hbase.thrift2.generated.TBloomFilterImpl.findByValue(iprot.readI32());
+ struct.setBloomFilterImplIsSet(true);
+ } else {
+ org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+ }
+ break;
default:
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
}
@@ -2226,6 +2332,13 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnFamilyDescr
oprot.writeBool(struct.inMemory);
oprot.writeFieldEnd();
}
+ if (struct.bloomFilterImpl != null) {
+ if (struct.isSetBloomFilterImpl()) {
+ oprot.writeFieldBegin(BLOOM_FILTER_IMPL_FIELD_DESC);
+ oprot.writeI32(struct.bloomFilterImpl.getValue());
+ oprot.writeFieldEnd();
+ }
+ }
oprot.writeFieldStop();
oprot.writeStructEnd();
}
@@ -2302,7 +2415,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TColumnFamilyDescri
if (struct.isSetInMemory()) {
optionals.set(18);
}
- oprot.writeBitSet(optionals, 19);
+ if (struct.isSetBloomFilterImpl()) {
+ optionals.set(19);
+ }
+ oprot.writeBitSet(optionals, 20);
if (struct.isSetAttributes()) {
{
oprot.writeI32(struct.attributes.size());
@@ -2374,6 +2490,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TColumnFamilyDescri
if (struct.isSetInMemory()) {
oprot.writeBool(struct.inMemory);
}
+ if (struct.isSetBloomFilterImpl()) {
+ oprot.writeI32(struct.bloomFilterImpl.getValue());
+ }
}
@Override
@@ -2381,7 +2500,7 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TColumnFamilyDescrip
org.apache.thrift.protocol.TTupleProtocol iprot = (org.apache.thrift.protocol.TTupleProtocol) prot;
struct.name = iprot.readBinary();
struct.setNameIsSet(true);
- java.util.BitSet incoming = iprot.readBitSet(19);
+ java.util.BitSet incoming = iprot.readBitSet(20);
if (incoming.get(0)) {
{
org.apache.thrift.protocol.TMap _map154 = iprot.readMapBegin(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING);
@@ -2480,6 +2599,10 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TColumnFamilyDescrip
struct.inMemory = iprot.readBool();
struct.setInMemoryIsSet(true);
}
+ if (incoming.get(19)) {
+ struct.bloomFilterImpl = org.apache.hadoop.hbase.thrift2.generated.TBloomFilterImpl.findByValue(iprot.readI32());
+ struct.setBloomFilterImplIsSet(true);
+ }
}
}
diff --git a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
index 43cf01f5b56e..b6c64c46f3ba 100644
--- a/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
+++ b/hbase-thrift/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
@@ -339,7 +339,22 @@ enum TBloomFilterType {
/**
* Bloom enabled with Table row prefix as Key, specify the length of the prefix
*/
- ROWPREFIX_FIXED_LENGTH = 3,
+ ROWPREFIX_FIXED_LENGTH = 3
+}
+
+/**
+ * Thrift wrapper around
+ * org.apache.hadoop.hbase.regionserver.BloomFilterImpl
+ */
+enum TBloomFilterImpl {
+ /**
+ * Traditional Bloom filter implementation
+ */
+ BLOOM = 0,
+ /**
+ * Ribbon filter implementation (more space-efficient than Bloom filters)
+ */
+ RIBBON = 1
}
/**
@@ -432,6 +447,7 @@ struct TColumnFamilyDescriptor {
18: optional bool compressTags
19: optional bool evictBlocksOnClose
20: optional bool inMemory
+ 21: optional TBloomFilterImpl bloomFilterImpl
}
diff --git a/src/main/asciidoc/_chapters/performance.adoc b/src/main/asciidoc/_chapters/performance.adoc
index 170c047c139d..502d3d7dd993 100644
--- a/src/main/asciidoc/_chapters/performance.adoc
+++ b/src/main/asciidoc/_chapters/performance.adoc
@@ -379,6 +379,7 @@ The following example creates a table and enables a ROWCOL Bloom filter on the `
hbase> create 'mytable',{NAME => 'colfam1', BLOOMFILTER => 'ROWCOL'}
----
+[[bloom.server.config]]
==== Configuring Server-Wide Behavior of Bloom Filters
You can configure the following settings in the _hbase-site.xml_.
@@ -404,10 +405,6 @@ You can configure the following settings in the _hbase-site.xml_.
| The guaranteed maximum fold rate. Changing this setting should not be
necessary and is not recommended.
-| io.storefile.bloom.max.keys
-| 128000000
-| For default (single-block) Bloom filters, this specifies the maximum number of keys.
-
| io.storefile.delete.family.bloom.enabled
| true
| Master switch to enable Delete Family Bloom filters and store them in the StoreFile.
@@ -422,6 +419,39 @@ You can configure the following settings in the _hbase-site.xml_.
| Enables cache-on-write for inline blocks of a compound Bloom filter.
|===
+[[ribbon.filter]]
+==== Ribbon Filter
+
+link:https://arxiv.org/abs/2103.02515[Ribbon Filter] is a space-efficient alternative to Bloom Filter, achieving approximately 30% space savings while maintaining comparable query performance.
+It was introduced in link:https://issues.apache.org/jira/browse/HBASE-29842[HBASE-29842].
+
+* Bloom Filter requires ~9.6 bits/key for 1% FPR (44% overhead vs theoretical minimum)
+* Ribbon Filter achieves ~7.3 bits/key for 1% FPR (~10% overhead)
+
+===== Enabling Ribbon Filter
+
+You can enable Ribbon Filter by setting `BLOOMFILTER_IMPL` to `RIBBON`:
+
+----
+hbase> create 'mytable',{NAME => 'colfam1', BLOOMFILTER => 'ROW', BLOOMFILTER_IMPL => 'RIBBON'}
+----
+
+===== Configuring Global Default
+
+A global default can be set in _hbase-site.xml_ (case insensitive):
+
+[source,xml]
+----
+
+ io.storefile.bloom.filter.impl
+ ribbon
+
+----
+
+When both global and per-table settings exist, the per-table setting takes precedence.
+
+Ribbon Filter uses existing Bloom Filter settings. See <> for details.
+
[[schema.cf.blocksize]]
=== ColumnFamily BlockSize