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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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 @@ -219,6 +223,79 @@ 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>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>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 @@ -112,6 +112,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 @@ -93,6 +93,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(byte[] buffer, int offset, int length) throws IOException {
// No call to this.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,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 @@ -123,6 +124,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 @@ -496,11 +498,18 @@ private boolean nextInternal(List<Cell> results, ScannerContext 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().
Cell 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 @@ -742,6 +751,49 @@ 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).
*/
private boolean nextRowViaHint(ScannerContext scannerContext, Cell curRowCell, Cell 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) {
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; otherwise returns {@code null}.
*/
private Cell getHintForRejectedRow(Cell rowCell) throws IOException {
if (filter == null) {
return null;
}
return filter.getHintForRejectedRow(rowCell);
}

protected boolean shouldStop(Cell currentRowCell) {
if (currentRowCell == null) {
return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,18 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {

private Cell curColCell = null;

/**
* Holds a seek-hint produced by {@link org.apache.hadoop.hbase.filter.Filter#getSkipHint(Cell)}
* at one of the structural short-circuit points in {@link #matchColumn}. When non-null this is
* returned by {@link #getNextKeyHint} instead of delegating to
* {@link org.apache.hadoop.hbase.filter.Filter#getNextCellHint}, because the hint was computed
* for a cell that never reached {@code filterCell}. Cleared on every {@link #getNextKeyHint} call
* so it cannot leak across multiple seek-hint cycles.
*/
private Cell pendingSkipHint = null;

private final boolean reversed;

private static Cell createStartKey(Scan scan, ScanInfo scanInfo) {
if (scan.includeStartRow()) {
return createStartKeyFromRow(scan.getStartRow(), scanInfo);
Expand All @@ -80,6 +92,7 @@ protected UserScanQueryMatcher(Scan scan, ScanInfo scanInfo, ColumnTracker colum
this.versionsAfterFilter = 0;
}
this.stopRow = scan.getStopRow();
this.reversed = scan.isReversed();
TimeRange timeRange = scan.getColumnFamilyTimeRange().get(scanInfo.getFamily());
if (timeRange == null) {
this.tr = scan.getTimeRange();
Expand All @@ -105,6 +118,11 @@ public Filter getFilter() {

@Override
public Cell getNextKeyHint(Cell cell) throws IOException {
if (pendingSkipHint != null) {
Cell hint = pendingSkipHint;
pendingSkipHint = null;
return hint;
}
if (filter == null) {
return null;
} else {
Expand All @@ -118,20 +136,47 @@ public void beforeShipped() throws IOException {
if (curColCell != null) {
this.curColCell = KeyValueUtil.toNewKeyCell(this.curColCell);
}
if (pendingSkipHint != null) {
this.pendingSkipHint = KeyValueUtil.toNewKeyCell(this.pendingSkipHint);
}
}

@Override
public void setToNewRow(Cell currentRow) {
pendingSkipHint = null;
super.setToNewRow(currentRow);
}

@Override
public void clearCurrentRow() {
pendingSkipHint = null;
super.clearCurrentRow();
}

// At each structural short-circuit below (time-range, column-exclusion, version-exhaustion),
// the filter is consulted via resolveSkipHint() before falling back to the default skip/seek
// code. This lets filters provide a forward seek target even when filterCell is never called.
protected final MatchCode matchColumn(Cell cell, long timestamp, byte typeByte)
throws IOException {
int tsCmp = tr.compare(timestamp);
if (tsCmp > 0) {
if (resolveSkipHint(cell)) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
return MatchCode.SKIP;
}
if (tsCmp < 0) {
if (resolveSkipHint(cell)) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
return columns.getNextRowOrNextColumn(cell);
}
// STEP 1: Check if the column is part of the requested columns
MatchCode matchCode = columns.checkColumn(cell, typeByte);
if (matchCode != MatchCode.INCLUDE) {
if (resolveSkipHint(cell)) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
return matchCode;
}
/*
Expand All @@ -141,8 +186,14 @@ protected final MatchCode matchColumn(Cell cell, long timestamp, byte typeByte)
matchCode = columns.checkVersions(cell, timestamp, typeByte, false);
switch (matchCode) {
case SKIP:
if (resolveSkipHint(cell)) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
return MatchCode.SKIP;
case SEEK_NEXT_COL:
if (resolveSkipHint(cell)) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
return MatchCode.SEEK_NEXT_COL;
default:
// It means it is INCLUDE, INCLUDE_AND_SEEK_NEXT_COL or INCLUDE_AND_SEEK_NEXT_ROW.
Expand All @@ -156,6 +207,31 @@ protected final MatchCode matchColumn(Cell cell, long timestamp, byte typeByte)
: mergeFilterResponse(cell, matchCode, filter.filterCell(cell));
}

/**
* Asks the current filter for a seek hint via
* {@link org.apache.hadoop.hbase.filter.Filter#getSkipHint(Cell)}, validates the returned cell,
* and if non-null stores it in {@link #pendingSkipHint} so that {@link #getNextKeyHint} can
* return it when the scan pipeline asks for the seek target after receiving
* {@link ScanQueryMatcher.MatchCode#SEEK_NEXT_USING_HINT}.
*/
private boolean resolveSkipHint(Cell cell) throws IOException {
if (filter == null) {
return false;
}
Cell raw = filter.getSkipHint(cell);
if (raw == null) {
return false;
}
// Full-key compare is intentional: skip hints can advance within the same row
// (e.g., to a later column), not just across rows.
int cmp = rowComparator.compare(raw, cell);
if ((!reversed && cmp <= 0) || (reversed && cmp >= 0)) {
return false;
}
pendingSkipHint = raw;
return true;
}

/**
* Call this when scan has filter. Decide the desired behavior by checkVersions's MatchCode and
* filterCell's ReturnCode. Cell may be skipped by filter, so the column versions in result may be
Expand Down
Loading