@@ -60,7 +60,8 @@ public class TestCrossCFSeekHint {
6060
6161 private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil ();
6262
63- private static final byte [] ROW = Bytes .toBytes ("row" );
63+ private static final byte [] ROW1 = Bytes .toBytes ("row1" );
64+ private static final byte [] ROW2 = Bytes .toBytes ("row2" );
6465 private static final byte [] QUAL = Bytes .toBytes ("q" );
6566 private static final int NUM_CELLS = 100 ;
6667
@@ -87,18 +88,29 @@ static Stream<Arguments> params() {
8788 }
8889
8990 /**
90- * Filter that returns SEEK_NEXT_USING_HINT for cells not in the target family, with a hint
91- * pointing to the target family. Counts how many times filterCell is called so we can verify the
92- * scanner didn't traverse all cells.
91+ * Filter that INCLUDEs cells from the data family up to a threshold qualifier, then returns
92+ * SEEK_NEXT_USING_HINT with a hint pointing to the target family. All target family cells are
93+ * INCLUDEd. Counts filterCell calls to verify no cell-by-cell traversal.
94+ * <p>
95+ * This simulates a filter that wants some cells from CF1, then skips ahead to CF2. If the fix
96+ * permanently closes CF1's store scanner, CF1 cells would be missing on subsequent rows.
9397 */
9498 public static class CrossCFHintFilter extends FilterBase {
9599 private final byte [] targetFamily ;
96100 private final byte [] targetQualifier ;
101+ private final byte [] hintThreshold ;
97102 private long filterCellCount ;
98103
99- public CrossCFHintFilter (byte [] targetFamily , byte [] targetQualifier ) {
104+ /**
105+ * @param targetFamily the CF to hint/seek to
106+ * @param targetQualifier qualifier for the hint cell
107+ * @param hintThreshold when a data CF cell's qualifier >= this, return SEEK_NEXT_USING_HINT
108+ * instead of INCLUDE. Cells before this threshold are INCLUDEd.
109+ */
110+ public CrossCFHintFilter (byte [] targetFamily , byte [] targetQualifier , byte [] hintThreshold ) {
100111 this .targetFamily = targetFamily ;
101112 this .targetQualifier = targetQualifier ;
113+ this .hintThreshold = hintThreshold ;
102114 }
103115
104116 @ Override
@@ -107,6 +119,13 @@ public ReturnCode filterCell(Cell c) throws IOException {
107119 if (CellUtil .matchingFamily (c , targetFamily )) {
108120 return ReturnCode .INCLUDE ;
109121 }
122+ // Data CF cell: include if qualifier < threshold, otherwise hint to target CF
123+ if (
124+ Bytes .compareTo (c .getQualifierArray (), c .getQualifierOffset (), c .getQualifierLength (),
125+ hintThreshold , 0 , hintThreshold .length ) < 0
126+ ) {
127+ return ReturnCode .INCLUDE ;
128+ }
110129 return ReturnCode .SEEK_NEXT_USING_HINT ;
111130 }
112131
@@ -136,34 +155,52 @@ public void testCrossCFSeekHint(byte[] dataCF, byte[] targetCF, boolean flush, b
136155 TEST_UTIL .getConfiguration (), td );
137156
138157 try {
139- for (int i = 0 ; i < NUM_CELLS ; i ++) {
140- Put p = new Put (ROW );
141- p .addColumn (dataCF , Bytes .toBytes (String .format ("q%05d" , i )), Bytes .toBytes ("v" ));
158+ // Write two rows. Each row has NUM_CELLS qualifiers in dataCF (q00000..q00099)
159+ // and one cell in targetCF.
160+ for (byte [] row : new byte [][] { ROW1 , ROW2 }) {
161+ for (int i = 0 ; i < NUM_CELLS ; i ++) {
162+ Put p = new Put (row );
163+ p .addColumn (dataCF , Bytes .toBytes (String .format ("q%05d" , i )), Bytes .toBytes ("v" ));
164+ region .put (p );
165+ }
166+ Put p = new Put (row );
167+ p .addColumn (targetCF , QUAL , Bytes .toBytes ("target" ));
142168 region .put (p );
143169 }
144- Put p = new Put (ROW );
145- p .addColumn (targetCF , QUAL , Bytes .toBytes ("target" ));
146- region .put (p );
147170 if (flush ) {
148171 region .flush (true );
149172 }
150173
151- CrossCFHintFilter filter = new CrossCFHintFilter (targetCF , QUAL );
174+ // The filter INCLUDEs dataCF cells with qualifier < "q00002" (i.e. q00000, q00001),
175+ // then returns SEEK_NEXT_USING_HINT at q00002 with a hint to targetCF.
176+ // This means each row should return: dataCF:q00000, dataCF:q00001, targetCF:q.
177+ byte [] hintThreshold = Bytes .toBytes ("q00002" );
178+ int includedPerRow = 2 ; // q00000, q00001
179+ CrossCFHintFilter filter = new CrossCFHintFilter (targetCF , QUAL , hintThreshold );
152180 Scan scan = new Scan ();
153181 scan .setFilter (filter );
154182 scan .setReversed (reversed );
155183
156- List <Cell > results = new ArrayList <>();
184+ List <Cell > row1Results = new ArrayList <>();
185+ List <Cell > row2Results = new ArrayList <>();
157186 try (RegionScanner scanner = region .getScanner (scan )) {
158- scanner .next (results );
187+ boolean hasMore = scanner .next (row1Results );
188+ assertTrue (hasMore , "Should have more rows after first row" );
189+ scanner .next (row2Results );
159190 }
160191
161- assertEquals (1 , results .size (), "Should return the cell from target CF" );
162- assertTrue (CellUtil .matchingFamily (results .get (0 ), targetCF ),
163- "Result should be from target CF" );
192+ // First row: includedPerRow data CF cells + 1 target CF cell
193+ assertEquals (includedPerRow + 1 , row1Results .size (), "First row result count" );
194+
195+ // Second row: if the data CF's store scanner was permanently closed on the
196+ // first row, we would lose the data CF cells here.
197+ assertEquals (includedPerRow + 1 , row2Results .size (),
198+ "Second row must also return data CF cells; store scanner must survive cross-CF hint" );
164199
165- // 1 call for the first data CF cell (hint triggers close) + 1 for the target CF cell
166- assertEquals (2 , filter .getFilterCellCount (),
200+ // Verify no cell-by-cell traversal of the remaining 98 data CF cells per row.
201+ // Per row: includedPerRow INCLUDEs + 1 SEEK_NEXT_USING_HINT + 1 target CF INCLUDE.
202+ int callsPerRow = includedPerRow + 1 + 1 ;
203+ assertEquals (callsPerRow * 2 , filter .getFilterCellCount (),
167204 "Cross-CF hint should not cause cell-by-cell traversal (HBASE-28902)" );
168205 } finally {
169206 HBaseTestingUtil .closeRegionAndWAL (region );
0 commit comments