HBASE-29974: Persist filter hints across scan circuit breaks#7882
HBASE-29974: Persist filter hints across scan circuit breaks#7882shubham-roy wants to merge 8 commits intoapache:masterfrom
Conversation
…rcuit breaks in scan pipeline, causing unnecessary cell-level iteration.
93eb1f0 to
bc24248
Compare
|
@shubham-roy could you rebase with latest master? |
|
FYI @tkhurana |
|
|
|
@virajjasani , I have resolved merge conflicts and have also applied spotless command. For the test failures: The above passes in my local. Are we sure it is not a flapper? Also, I believe the 3 count of flakes can be ignored, right? |
virajjasani
left a comment
There was a problem hiding this comment.
Left few comments, looks good overall
| } | ||
| 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()); | ||
| } |
There was a problem hiding this comment.
Is this part changing at all? I guess this is only structural change.
Can we remove this change just to keep it like it was before?
| // ----------------------------------------------------------------------- | ||
| // Tests for HBASE-29974: Filter#getSkipHint consulted at matchColumn | ||
| // structural short-circuits (time-range, column, and version gates). | ||
| // ----------------------------------------------------------------------- |
There was a problem hiding this comment.
Let's remove these lines
| @Test | ||
| public void testHintJumpsOverAllRejectedRowsInOneSingleSeek() throws IOException { |
There was a problem hiding this comment.
testHintJumpsOverAllRejectedRowsInOneSingleSeek() and testNullHintFallsThroughToLegacyNextRowBehaviour() should be combined in single test. When we check for data correctness, we should check it on the same table, same data, one with hintFilter, one with noHintFilter and compare cells. Separate tests do not serve the same purpose.
| * </ol> | ||
| */ | ||
| @Category({ FilterTests.class, MediumTests.class }) | ||
| public class TestFilterHintForRejectedRow { |
There was a problem hiding this comment.
For all tests in this class, it seems we are not overriding getSkipHint(). Let's create one test where we some cells are out of scan time-range and getSkipHint() is used. Then for the same data, we can use scan without getSkipHint() and it should still return same cells, for the data correctness comparison.
There was a problem hiding this comment.
Though we need both getHintForRejectedRow() and getSkipHint() implementations,
for phoenix cdc, i believe getSkipHint() will more likely be common hint than getHintForRejectedRow(), is that correct?
There was a problem hiding this comment.
getHintForRejectedRow() is also equally applicable for the phoenix cdc based backup solution that we are developing. We identify row keys that were mutated for the backup time range via CDC and then get information for the same row keys. While skipping row keys that were not mutated getHintForRejectedRow() comes into play.

JIRA
HBASE-29974
Problem / Motivation
HBase's scan pipeline applies several structural short-circuit checks in
UserScanQueryMatcher.matchColumn— time-range gate, column-set exclusion, andversion-limit exhaustion — before
Filter.filterCellis ever called. When anyof these gates fire, the pipeline returns a plain
SKIPorSEEK_NEXT_COLcodeand the filter is bypassed entirely.
Similarly,
RegionScannerImpliterates through every cell in a rejected rowone-by-one via
nextRow()afterfilterRowKeyreturnstrue, even though afilter can deterministically compute the next valid
row boundary.
The consequence is that filter implementations that could provide meaningful
seek hints are never consulted at these decision points, forcing the scanner to
advance cell-by-cell rather than issuing a single
requestSeek. Forrange-oriented or sparse-column filters over large tables this results in
significant, avoidable read amplification.
Root Cause
The existing
getNextCellHint/SEEK_NEXT_USING_HINTmechanism requiresfilterCellto have been called first; there was no API contract or call-sitefor filters to provide a forward hint when the cell was structurally excluded
before reaching
filterCell. The three structural gates inmatchColumnand thefilterRowKeyrejection path all lacked hook points.Solution
New public API (
Filter)Two new optional methods are added to the abstract
Filterclass, both withbackward-compatible
nulldefaults inFilterBaseandFilterWrapper:getSkipHint(Cell skippedCell)filterCellgetHintForRejectedRow(Cell firstRowCell)filterRowKeybefore any cell-level iterationfilterRowKeyUserScanQueryMatcher(server-side)pendingSkipHintfield andresolveSkipHint()helper.tsCmp > 0,time-range
tsCmp < 0, column-set exclusion, versionSKIP/SEEK_NEXT_COL),calls
resolveSkipHint()and promotes the result toSEEK_NEXT_USING_HINTwhen a non-null hint is returned.
getNextKeyHintdrainspendingSkipHintfirst, before delegating tofilter.getNextCellHint, ensuring the correct hint is returned for the cellthat triggered the structural gate.
RegionScannerImpl(server-side)getHintForRejectedRow(Cell)validates and retrieves the filter hint after afilterRowKeyrejection.nextRowViaHint(ScannerContext, Cell, ExtendedCell)replaces thenextRow()call with a single
storeHeap.requestSeek(hint), wrapped withskippingRowmode to keep block-size accounting consistent.
Backward Compatibility
Filtermethods returnnullby default(defined on
Filterdirectly, with matching no-ops inFilterBaseandFilterWrapper), so all existing filter implementations continue to behaveidentically.
DoNotRetryIOExceptionis thrown (consistent with existinggetNextCellHintvalidation) if a filter returns a non-
ExtendedCellfrom either new method.