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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,75 @@ 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>
* </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>
* </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,27 @@ 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 @@ -29,6 +29,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 @@ -502,7 +503,12 @@ private boolean nextInternal(List<? super ExtendedCell> results, ScannerContext
// 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
incrementCountOfRowsScannedMetric(scannerContext);
boolean moreRows = nextRow(scannerContext, current);
// 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);
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 @@ -744,6 +750,72 @@ 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
* {@code requestSeek} to jump directly to the filter's suggested position.
*
* <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.
*
* @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.";

// Enable skipping-row mode so block-size accounting is consistent with nextRow().
scannerContext.setSkippingRow(true);
this.storeHeap.requestSeek(hint, true, true);
scannerContext.setSkippingRow(false);

resetFilters();

// Notify coprocessors, identical to the epilogue in nextRow().
return this.region.getCoprocessorHost() == null
|| this.region.getCoprocessorHost().postScannerFilterRow(this, 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
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,16 @@ public abstract class UserScanQueryMatcher extends ScanQueryMatcher {

private ExtendedCell 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 ExtendedCell pendingSkipHint = null;

private static ExtendedCell createStartKey(Scan scan, ScanInfo scanInfo) {
if (scan.includeStartRow()) {
return createStartKeyFromRow(scan.getStartRow(), scanInfo);
Expand Down Expand Up @@ -107,18 +117,26 @@ public Filter getFilter() {

@Override
public ExtendedCell getNextKeyHint(ExtendedCell cell) throws IOException {
// A structural short-circuit in matchColumn (time-range, column, or version gate) may have
// stored a hint via resolveSkipHint() before returning SEEK_NEXT_USING_HINT. Drain and return
// it first; it takes priority because it was produced for the exact cell that triggered the
// seek code, without ever calling filterCell.
if (pendingSkipHint != null) {
ExtendedCell hint = pendingSkipHint;
pendingSkipHint = null;
return hint;
}
// Normal path: filterCell returned SEEK_NEXT_USING_HINT — delegate to the filter.
if (filter == null) {
return null;
}
Cell hint = filter.getNextCellHint(cell);
if (hint == null || hint instanceof ExtendedCell) {
return (ExtendedCell) hint;
} else {
Cell hint = filter.getNextCellHint(cell);
if (hint == null || hint instanceof ExtendedCell) {
return (ExtendedCell) hint;
} else {
throw new DoNotRetryIOException("Incorrect filter implementation, "
+ "the Cell returned by getNextKeyHint is not an ExtendedCell. Filter class: "
+ filter.getClass().getName());
}

throw new DoNotRetryIOException("Incorrect filter implementation, "
+ "the Cell returned by getNextKeyHint is not an ExtendedCell. Filter class: "
+ filter.getClass().getName());
}
}

Expand All @@ -134,14 +152,40 @@ protected final MatchCode matchColumn(ExtendedCell cell, long timestamp, byte ty
throws IOException {
int tsCmp = tr.compare(timestamp);
if (tsCmp > 0) {
// Cell is newer than the scan's time-range upper bound. Give the filter one last chance to
// provide a seek hint before we fall back to a plain cell-level SKIP. This addresses
// HBASE-29974 Path 2: time-range gate fires before filterCell is reached.
if (filter != null) {
ExtendedCell hint = resolveSkipHint(cell);
if (hint != null) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
}
return MatchCode.SKIP;
}
if (tsCmp < 0) {
// Cell is older than the scan's time-range lower bound. Give the filter a chance to provide
// a seek hint before we defer to the column tracker's next-row/next-column suggestion.
// Addresses HBASE-29974 Path 2: time-range gate fires before filterCell is reached.
if (filter != null) {
ExtendedCell hint = resolveSkipHint(cell);
if (hint != null) {
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) {
// Column is excluded by the scan's column set. Give the filter a chance to provide a
// seek hint before the column-tracker's suggestion is used. Addresses HBASE-29974 Path 3.
if (filter != null) {
ExtendedCell hint = resolveSkipHint(cell);
if (hint != null) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
}
return matchCode;
}
/*
Expand All @@ -151,8 +195,24 @@ protected final MatchCode matchColumn(ExtendedCell cell, long timestamp, byte ty
matchCode = columns.checkVersions(cell, timestamp, typeByte, false);
switch (matchCode) {
case SKIP:
// Version limit reached; skip this cell. Give the filter a hint opportunity before
// falling back to SKIP. Addresses HBASE-29974 Path 3: version gate fires before filterCell.
if (filter != null) {
ExtendedCell hint = resolveSkipHint(cell);
if (hint != null) {
return MatchCode.SEEK_NEXT_USING_HINT;
}
}
return MatchCode.SKIP;
case SEEK_NEXT_COL:
// Version limit reached; advance to the next column. Give the filter a hint opportunity
// before falling back to SEEK_NEXT_COL. Addresses HBASE-29974 Path 3.
if (filter != null) {
ExtendedCell hint = resolveSkipHint(cell);
if (hint != null) {
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 @@ -166,6 +226,35 @@ protected final MatchCode matchColumn(ExtendedCell cell, long timestamp, byte ty
: mergeFilterResponse(cell, matchCode, filter.filterCell(cell));
}

/**
* Calls {@link org.apache.hadoop.hbase.filter.Filter#getSkipHint(Cell)} on the current filter,
* validates the returned cell type, and stores it as {@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}.
*
* <p>This is only called from the structural short-circuit branches of {@link #matchColumn},
* where {@code filterCell} has <em>not</em> been called, in accordance with the stateless
* contract of {@code Filter#getSkipHint}.
*
* @param cell the cell that triggered the structural short-circuit
* @return the validated {@link ExtendedCell} hint, or {@code null} if the filter returns 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 resolveSkipHint(ExtendedCell cell) throws IOException {
Cell raw = filter.getSkipHint(cell);
if (raw == null) {
return null;
}
if (!(raw instanceof ExtendedCell)) {
throw new DoNotRetryIOException(
"Incorrect filter implementation: the Cell returned by getSkipHint "
+ "is not an ExtendedCell. Filter class: " + filter.getClass().getName());
}
pendingSkipHint = (ExtendedCell) raw;
return pendingSkipHint;
}

/**
* 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