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 @@ -148,6 +148,14 @@ public Cell getNextCellHint(Cell cell) {
return PrivateCellUtil.createFirstOnRowCol(cell, prefix, 0, prefix.length);
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
if (this.prefix == null) {
return null;
}
return getNextCellHint(skippedCell);
}

@Override
public String toString() {
return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,14 @@ public Cell getNextCellHint(Cell cell) {
return PrivateCellUtil.createFirstOnRowCol(cell, this.minColumn, 0, len(this.minColumn));
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
if (this.minColumn == null) {
return null;
}
return getNextCellHint(skippedCell);
}

@Override
public String toString() {
return this.getClass().getSimpleName() + " " + (this.minColumnInclusive ? "[" : "(")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -216,9 +216,13 @@ public enum ReturnCode {
* 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>
* <li><strong>Composite filter support:</strong> {@code FilterList} (both {@code MUST_PASS_ALL}
* and {@code MUST_PASS_ONE}), {@code SkipFilter}, and {@code WhileMatchFilter} delegate this
* method to their sub-filters and merge the results. For AND ({@code MUST_PASS_ALL}), only
* sub-filters whose {@code filterRowKey} individually returned {@code true} are consulted, and
* the farthest (maximal-step) hint among them is returned. For OR ({@code MUST_PASS_ONE}), the
* nearest hint is returned only when every non-terminated sub-filter provides one — any null
* collapses the OR result to null.</li>
* </ul>
* @param firstRowCell the first cell encountered in the rejected row; contains the row key that
* was passed to {@code filterRowKey}
Expand Down Expand Up @@ -255,9 +259,11 @@ public Cell getHintForRejectedRow(final Cell firstRowCell) throws IOException {
* <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>
* <li><strong>Composite filter support:</strong> {@code FilterList} (both {@code MUST_PASS_ALL}
* and {@code MUST_PASS_ONE}), {@code SkipFilter}, and {@code WhileMatchFilter} delegate this
* method to their sub-filters and merge the results (maximal step for AND; for OR, the nearest
* hint is returned only when every non-terminated sub-filter provides one — any null collapses
* the OR result to null).</li>
* </ul>
* @param skippedCell the cell that was rejected by the time-range, column, or version gate before
* {@code filterCell} could be consulted
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -242,6 +242,16 @@ public Cell getNextCellHint(Cell currentCell) throws IOException {
return this.filterListBase.getNextCellHint(currentCell);
}

@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
return this.filterListBase.getHintForRejectedRow(firstRowCell);
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
return this.filterListBase.getSkipHint(skippedCell);
}

@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return this.filterListBase.isFamilyEssential(name);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
Expand All @@ -34,13 +35,20 @@ public class FilterListWithAND extends FilterListBase {

private List<Filter> seekHintFilters = new ArrayList<>();
private boolean[] hintingFilters;
/**
* Tracks which sub-filters returned {@code true} from {@link Filter#filterRowKey(Cell)}. Set in
* {@code filterRowKey()}, consumed by {@code getHintForRejectedRow()}, cleared only by
* {@code reset()} — callers must invoke {@code reset()} between rows to avoid stale state.
*/
private boolean[] rejectedByFilterRowKey;

public FilterListWithAND(List<Filter> filters) {
super(filters);
// For FilterList with AND, when call FL's transformCell(), we should transform cell for all
// sub-filters (because all sub-filters return INCLUDE*). So here, fill this array with true. we
// keep this in FilterListWithAND for abstracting the transformCell() in FilterListBase.
subFiltersIncludedCell = new ArrayList<>(Collections.nCopies(filters.size(), true));
rejectedByFilterRowKey = new boolean[filters.size()];
cacheHintingFilters();
}

Expand All @@ -51,6 +59,7 @@ public void addFilterLists(List<Filter> filters) {
}
this.filters.addAll(filters);
this.subFiltersIncludedCell.addAll(Collections.nCopies(filters.size(), true));
this.rejectedByFilterRowKey = Arrays.copyOf(this.rejectedByFilterRowKey, this.filters.size());
this.cacheHintingFilters();
}

Expand Down Expand Up @@ -237,6 +246,7 @@ public void reset() throws IOException {
filters.get(i).reset();
}
seekHintFilters.clear();
Arrays.fill(rejectedByFilterRowKey, false);
}

@Override
Expand All @@ -258,6 +268,7 @@ public boolean filterRowKey(Cell firstRowCell) throws IOException {
// will catch the row changed event by filterRowKey(). If we return early here, those
// filters will have no chance to update their row state.
anyRowKeyFiltered = true;
rejectedByFilterRowKey[i] = true;
} else if (hintingFilters[i]) {
// If filterRowKey returns false and this is a hinting filter, then we must not filter this
// rowkey.
Expand Down Expand Up @@ -318,6 +329,60 @@ public Cell getNextCellHint(Cell currentCell) throws IOException {
return maxHint;
}

/**
* Maximal step: return the farthest hint among sub-filters that actually rejected the row. Only
* sub-filters whose {@link Filter#filterRowKey(Cell)} returned {@code true} are consulted,
* honouring the per-filter contract. Null hints are ignored; if no rejecting sub-filter provides
* a hint, return null.
*/
@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
if (isEmpty()) {
return super.getHintForRejectedRow(firstRowCell);
}
Cell maxHint = null;
for (int i = 0, n = filters.size(); i < n; i++) {
if (!rejectedByFilterRowKey[i]) {
continue;
}
Filter filter = filters.get(i);
if (filter.filterAllRemaining()) {
continue;
}
Cell hint = filter.getHintForRejectedRow(firstRowCell);
if (hint == null) {
continue;
}
if (maxHint == null || this.compareCell(maxHint, hint) < 0) {
maxHint = hint;
}
}
return maxHint;
}

/** Maximal step: return the farthest skip hint among sub-filters. */
@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
if (isEmpty()) {
return super.getSkipHint(skippedCell);
}
Cell maxHint = null;
for (int i = 0, n = filters.size(); i < n; i++) {
Filter filter = filters.get(i);
if (filter.filterAllRemaining()) {
continue;
}
Cell hint = filter.getSkipHint(skippedCell);
if (hint == null) {
continue;
}
if (maxHint == null || this.compareCell(maxHint, hint) < 0) {
maxHint = hint;
}
}
return maxHint;
}

@Override
public boolean equals(Object obj) {
if (this == obj) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -392,6 +392,58 @@ public Cell getNextCellHint(Cell currentCell) throws IOException {
return minKeyHint;
}

/**
* Minimal step: return the nearest hint. If any non-terminated sub-filter returns null, the
* composite cannot safely skip, so return null.
*/
@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
if (isEmpty()) {
return super.getHintForRejectedRow(firstRowCell);
}
Cell minHint = null;
for (int i = 0, n = filters.size(); i < n; i++) {
Filter filter = filters.get(i);
if (filter.filterAllRemaining()) {
continue;
}
Cell hint = filter.getHintForRejectedRow(firstRowCell);
if (hint == null) {
return null;
}
if (minHint == null || this.compareCell(minHint, hint) > 0) {
minHint = hint;
}
}
return minHint;
}

/**
* Minimal step: return the nearest skip hint. Null from any sub-filter collapses the entire
* result to null.
*/
@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
if (isEmpty()) {
return super.getSkipHint(skippedCell);
}
Cell minHint = null;
for (int i = 0, n = filters.size(); i < n; i++) {
Filter filter = filters.get(i);
if (filter.filterAllRemaining()) {
continue;
}
Cell hint = filter.getSkipHint(skippedCell);
if (hint == null) {
return null;
}
if (minHint == null || this.compareCell(minHint, hint) > 0) {
minHint = hint;
}
}
return minHint;
}

@Override
public boolean equals(Object obj) {
if (this == obj) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -173,6 +173,29 @@ public Cell getNextCellHint(Cell cell) {
return PrivateCellUtil.createFirstOnRowCol(cell, hint, 0, hint.length);
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
if (sortedPrefixes.isEmpty()) {
return null;
}
byte[] qualifier = CellUtil.cloneQualifier(skippedCell);
TreeSet<byte[]> lesserOrEqual = (TreeSet<byte[]>) sortedPrefixes.headSet(qualifier, true);
byte[] target;
if (lesserOrEqual.isEmpty()) {
target = sortedPrefixes.first();
} else {
byte[] largest = lesserOrEqual.last();
if (Bytes.startsWith(qualifier, largest)) {
return null;
}
target = sortedPrefixes.higher(largest);
if (target == null) {
return null;
}
}
return PrivateCellUtil.createFirstOnRowCol(skippedCell, target, 0, target.length);
}

public TreeSet<byte[]> createTreeSet() {
return new TreeSet<>(new Comparator<Object>() {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,16 @@ boolean areSerializedFieldsEqual(Filter o) {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}

@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
return filter.getHintForRejectedRow(firstRowCell);
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
return filter.getSkipHint(skippedCell);
}

@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -139,6 +139,16 @@ boolean areSerializedFieldsEqual(Filter o) {
return getFilter().areSerializedFieldsEqual(other.getFilter());
}

@Override
public Cell getHintForRejectedRow(Cell firstRowCell) throws IOException {
return filter.getHintForRejectedRow(firstRowCell);
}

@Override
public Cell getSkipHint(Cell skippedCell) throws IOException {
return filter.getSkipHint(skippedCell);
}

@Override
public boolean isFamilyEssential(byte[] name) throws IOException {
return filter.isFamilyEssential(name);
Expand Down
Loading