Skip to content
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,10 @@
* <li>{@link #reset()} : reset the filter state before filtering a new row.</li>
* <li>{@link #filterAllRemaining()}: true means row scan is over; false means keep going.</li>
* <li>{@link #filterRowKey(Cell)}: true means drop this row; false means include.</li>
* <li>{@link #getHintForRejectedRow(Cell)}: if {@code filterRowKey} returned true, optionally
* provide a seek hint to skip past the rejected row efficiently.</li>
* <li>{@link #getSkipHint(Cell)}: when a cell is structurally skipped (time-range, column, or
* version gate) before {@code filterCell} is reached, optionally provide a seek hint.</li>
* <li>{@link #filterCell(Cell)}: decides whether to include or exclude this Cell. See
* {@link ReturnCode}.</li>
* <li>{@link #transformCell(Cell)}: if the Cell is included, let the filter transform the Cell.
Expand Down Expand Up @@ -188,6 +192,85 @@ public enum ReturnCode {
*/
abstract public Cell getNextCellHint(final Cell currentCell) throws IOException;

/**
* Provides a seek hint to bypass row-by-row scanning after {@link #filterRowKey(Cell)} rejects a
* row. When {@code filterRowKey} returns {@code true} the scan pipeline would normally iterate
* through every remaining cell in the rejected row one-by-one (via {@code nextRow()}) before
* moving on. If the filter can determine a better forward position — for example, the next range
* boundary in a {@code MultiRowRangeFilter} — it should return that target cell here, allowing
* the scanner to seek directly past the unwanted rows.
* <p>
* Contract:
* <ul>
* <li>Only called after {@link #filterRowKey(Cell)} has returned {@code true} for the same
* {@code firstRowCell}.</li>
* <li>Implementations may use state that was set during {@link #filterRowKey(Cell)} (e.g. an
* updated range pointer), but <strong>must not</strong> invoke {@link #filterCell(Cell)} logic —
* the caller guarantees that {@code filterCell} has not been called for this row.</li>
* <li>The returned {@link Cell}, if non-null, must be an
* {@link org.apache.hadoop.hbase.ExtendedCell} because filters are evaluated on the server
* side.</li>
* <li>Returning {@code null} (the default) falls through to the existing {@code nextRow()}
* behaviour, preserving full backward compatibility.</li>
* <li>For reversed scans ({@link org.apache.hadoop.hbase.client.Scan#isReversed()}), the hint
* must point to a <em>smaller</em> row key (earlier in reverse-scan direction). The scanner
* validates hint direction and falls back to {@code nextRow()} if the hint does not advance in
* the scan direction.</li>
* <li><strong>Composite filter limitation:</strong> {@code FilterList}, {@code SkipFilter}, and
* {@code WhileMatchFilter} do not currently delegate this method to wrapped sub-filters. Hints
* from filters used inside these wrappers will be silently ignored.</li>
* </ul>
* @param firstRowCell the first cell encountered in the rejected row; contains the row key that
* was passed to {@code filterRowKey}
* @return a {@link Cell} representing the earliest position the scanner should seek to, or
* {@code null} if this filter cannot provide a better position than a sequential skip
* @throws IOException in case an I/O or filter-specific failure needs to be signaled
* @see #filterRowKey(Cell)
*/
public Cell getHintForRejectedRow(final Cell firstRowCell) throws IOException {
return null;
}

/**
* Provides a seek hint for cells that are structurally skipped by the scan pipeline
* <em>before</em> {@link #filterCell(Cell)} is ever reached. The pipeline short-circuits on
* several criteria — time-range mismatch, column-set exclusion, and version-limit exhaustion —
* and in each case the filter is bypassed entirely. When an implementation can compute a
* meaningful forward position purely from the cell's coordinates (without needing the
* {@code filterCell} call sequence), it should return that position here so the scanner can seek
* ahead instead of advancing one cell at a time.
* <p>
* Contract:
* <ul>
* <li>May be called for cells that have <strong>never</strong> been passed to
* {@link #filterCell(Cell)}.</li>
* <li>Implementations <strong>must not</strong> modify any filter state; this method is treated
* as logically stateless. Only filters whose hint computation is based solely on immutable
* configuration (e.g. a fixed column range or a fuzzy-row pattern) should override this.</li>
* <li>The returned {@link Cell}, if non-null, must be an
* {@link org.apache.hadoop.hbase.ExtendedCell} because filters are evaluated on the server
* side.</li>
* <li>Returning {@code null} (the default) falls through to the existing structural skip/seek
* behaviour, preserving full backward compatibility.</li>
* <li>For reversed scans, the returned cell must have a <em>smaller</em> row key (i.e., earlier
* in reverse-scan direction) than the {@code skippedCell}. Hints that do not advance in the scan
* direction are silently ignored.</li>
* <li><strong>Composite filter limitation:</strong> {@code FilterList}, {@code SkipFilter}, and
* {@code WhileMatchFilter} do not currently delegate this method to wrapped sub-filters. Hints
* from filters used inside these wrappers will be silently ignored.</li>
* </ul>
* @param skippedCell the cell that was rejected by the time-range, column, or version gate before
* {@code filterCell} could be consulted
* @return a {@link Cell} representing the earliest position the scanner should seek to, or
* {@code null} if this filter cannot provide a better position than the structural hint
* @throws IOException in case an I/O or filter-specific failure needs to be signaled
* @see #filterCell(Cell)
* @see #getNextCellHint(Cell)
*/
public Cell getSkipHint(final Cell skippedCell) throws IOException {
return null;
}

/**
* Check that given column family is essential for filter to check row. Most filters always return
* true here. But some could have more sophisticated logic which could significantly reduce
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,28 @@ public Cell getNextCellHint(Cell currentCell) throws IOException {
return null;
}

/**
* Filters that cannot provide a seek hint after row-key rejection can inherit this no-op
* implementation. Subclasses whose row-key logic (e.g. a range pointer advanced inside
* {@link #filterRowKey(Cell)}) makes a better seek target available should override this.
* {@inheritDoc}
*/
@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
return null;
}

/**
* Filters that cannot provide a structural-skip seek hint can inherit this no-op implementation.
* Subclasses with purely configuration-driven, stateless hint computation (e.g. a fixed column
* range or fuzzy-row pattern) may override this to avoid cell-by-cell advancement when the
* time-range, column, or version gate fires. {@inheritDoc}
*/
@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
return null;
}

/**
* By default, we require all scan's column families to be present. Our subclasses may be more
* precise. {@inheritDoc}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,26 @@ public Cell getNextCellHint(Cell currentCell) throws IOException {
return this.filter.getNextCellHint(currentCell);
}

/**
* Delegates to the wrapped filter's {@link Filter#getHintForRejectedRow(Cell)} so that the scan
* pipeline can seek directly past a rejected row rather than iterating cell-by-cell via
* {@code nextRow()}. {@inheritDoc}
*/
@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
return this.filter.getHintForRejectedRow(firstRowCell);
}

/**
* Delegates to the wrapped filter's {@link Filter#getSkipHint(Cell)} so that the scan pipeline
* can seek ahead when a cell is structurally skipped before {@code filterCell} is reached.
* {@inheritDoc}
*/
@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
return this.filter.getSkipHint(skippedCell);
}

@Override
public boolean filterRowKey(Cell cell) throws IOException {
if (filterAllRemaining()) return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.ExtendedCell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
Expand Down Expand Up @@ -86,6 +87,7 @@ public class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {

protected final byte[] stopRow;
protected final boolean includeStopRow;
protected final boolean reversed;
protected final HRegion region;
protected final CellComparator comparator;

Expand Down Expand Up @@ -130,6 +132,7 @@ private static boolean hasNonce(HRegion region, long nonce) {
defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
this.stopRow = scan.getStopRow();
this.includeStopRow = scan.includeStopRow();
this.reversed = scan.isReversed();
this.operationId = scan.getId();

// synchronize on scannerReadPoints so that nobody calculates
Expand Down Expand Up @@ -504,11 +507,18 @@ private boolean nextInternal(List<? super ExtendedCell> results, ScannerContext
if (isFilterDoneInternal()) {
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
}
// HBASE-29974: ask the filter for a seek hint so we can jump directly past the rejected
// row instead of iterating through its cells one-by-one via nextRow().
ExtendedCell rowHint = getHintForRejectedRow(current);
// Typically the count of rows scanned is incremented inside #populateResult. However,
// here we are filtering a row based purely on its row key, preventing us from calling
// #populateResult. Thus, perform the necessary increment here to rows scanned metric
// #populateResult. Thus, perform the necessary increment here to rows scanned metric.
// Placed after getHintForRejectedRow so that a buggy filter throwing DNRIOE doesn't
// leave the metric incremented for a row that was never actually processed.
incrementCountOfRowsScannedMetric(scannerContext);
boolean moreRows = nextRow(scannerContext, current);
boolean moreRows = (rowHint != null)
? nextRowViaHint(scannerContext, current, rowHint)
: nextRow(scannerContext, current);
if (!moreRows) {
return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
}
Expand Down Expand Up @@ -750,6 +760,94 @@ protected boolean nextRow(ScannerContext scannerContext, Cell curRowCell) throws
|| this.region.getCoprocessorHost().postScannerFilterRow(this, curRowCell);
}

/**
* Fast-path alternative to {@link #nextRow} used when the filter has provided a seek hint via
* {@link org.apache.hadoop.hbase.filter.Filter#getHintForRejectedRow(Cell)}. Instead of iterating
* through every cell in the rejected row one-by-one, this method issues a single seek to jump
* directly to the filter's suggested position ({@code requestSeek} for forward scans,
* {@code backwardSeek} for reversed scans).
* <p>
* The skipping-row mode flag is set around the seek so that block-level size tracking continues
* to function (consistent with {@link #nextRow}), and the filter state is reset afterwards so the
* next row starts with a clean filter context.
* <p>
* <strong>Stop-row invariant:</strong> This method does not validate that {@code hint} falls
* within the scan's stop row. If the hint overshoots, the next iteration's
* {@link #shouldStop(Cell)} check catches it and returns NO_MORE_VALUES. One wasted seek may
* occur, but correctness is maintained.
* <p>
* <strong>Metrics note:</strong> The rows-scanned metric is incremented once by the caller for
* the rejected row. Rows physically skipped by the seek are not individually counted — this
* reflects the fact that no per-row work was done for those rows.
* <p>
* <strong>Coprocessor note:</strong> {@code postScannerFilterRow} is invoked once with
* {@code curRowCell}, not once per skipped row. Coprocessors counting filtered rows should be
* aware of this semantic when the hint path is used.
* @param scannerContext scanner context used for limit tracking
* @param curRowCell the first cell of the row that was rejected by {@code filterRowKey};
* passed to the coprocessor hook for observability
* @param hint the validated {@link ExtendedCell} returned by the filter; the scanner
* will seek to this position
* @return {@code true} if scanning should continue, {@code false} if a coprocessor requests an
* early stop (mirrors the contract of {@link #nextRow})
* @throws IOException if the seek or the coprocessor hook signals a failure
*/
private boolean nextRowViaHint(ScannerContext scannerContext, Cell curRowCell, ExtendedCell hint)
throws IOException {
assert this.joinedContinuationRow == null : "Trying to go to next row during joinedHeap read.";

int difference = comparator.compareRows(hint, curRowCell);
if ((!reversed && difference > 0) || (reversed && difference < 0)) {
scannerContext.setSkippingRow(true);
if (reversed) {
// ReversedKeyValueHeap does not support requestSeek; use backwardSeek
// to position at-or-before the hint within the target row.
// seekToPreviousRow would skip past the hint row entirely.
this.storeHeap.backwardSeek(hint);
} else {
this.storeHeap.requestSeek(hint, true, true);
}
scannerContext.setSkippingRow(false);

resetFilters();

return this.region.getCoprocessorHost() == null
|| this.region.getCoprocessorHost().postScannerFilterRow(this, curRowCell);
}

return nextRow(scannerContext, curRowCell);
}

/**
* Asks the current {@link org.apache.hadoop.hbase.filter.FilterWrapper} for a seek hint to use
* after a row has been rejected by {@link #filterRowKey}. If the wrapped filter overrides
* {@link org.apache.hadoop.hbase.filter.Filter#getHintForRejectedRow(Cell)}, this returns its
* answer as an {@link ExtendedCell}; otherwise returns {@code null}.
* <p>
* The returned cell is validated to be an {@link ExtendedCell} because filters run on the server
* side and the scanner infrastructure requires {@code ExtendedCell} references.
* @param rowCell the first cell of the rejected row (same cell passed to {@code filterRowKey})
* @return a validated {@link ExtendedCell} seek target, or {@code null} if the filter provides no
* hint
* @throws DoNotRetryIOException if the filter returns a non-{@link ExtendedCell} instance
* @throws IOException if the filter signals an I/O failure
*/
private ExtendedCell getHintForRejectedRow(Cell rowCell) throws IOException {
if (filter == null) {
return null;
}
Cell hint = filter.getHintForRejectedRow(rowCell);
if (hint == null) {
return null;
}
if (!(hint instanceof ExtendedCell)) {
throw new DoNotRetryIOException(
"Incorrect filter implementation: the Cell returned by getHintForRejectedRow "
+ "is not an ExtendedCell. Filter class: " + filter.getClass().getName());
}
return (ExtendedCell) hint;
}

protected boolean shouldStop(Cell currentRowCell) {
if (currentRowCell == null) {
return true;
Expand Down
Loading
Loading