001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.filter;
019
020import static org.junit.jupiter.api.Assertions.assertEquals;
021import static org.junit.jupiter.api.Assertions.assertFalse;
022import static org.junit.jupiter.api.Assertions.assertTrue;
023
024import java.io.IOException;
025import java.util.ArrayList;
026import java.util.Arrays;
027import java.util.List;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.CellComparator;
030import org.apache.hadoop.hbase.CellUtil;
031import org.apache.hadoop.hbase.CompareOperator;
032import org.apache.hadoop.hbase.HBaseTestingUtil;
033import org.apache.hadoop.hbase.HConstants;
034import org.apache.hadoop.hbase.KeyValue;
035import org.apache.hadoop.hbase.TableName;
036import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
037import org.apache.hadoop.hbase.client.Delete;
038import org.apache.hadoop.hbase.client.Durability;
039import org.apache.hadoop.hbase.client.Put;
040import org.apache.hadoop.hbase.client.RegionInfo;
041import org.apache.hadoop.hbase.client.RegionInfoBuilder;
042import org.apache.hadoop.hbase.client.Scan;
043import org.apache.hadoop.hbase.client.TableDescriptor;
044import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
045import org.apache.hadoop.hbase.filter.FilterList.Operator;
046import org.apache.hadoop.hbase.regionserver.HRegion;
047import org.apache.hadoop.hbase.regionserver.InternalScanner;
048import org.apache.hadoop.hbase.regionserver.RegionScanner;
049import org.apache.hadoop.hbase.testclassification.FilterTests;
050import org.apache.hadoop.hbase.testclassification.MediumTests;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.apache.hadoop.hbase.wal.WAL;
053import org.junit.jupiter.api.AfterEach;
054import org.junit.jupiter.api.BeforeEach;
055import org.junit.jupiter.api.Disabled;
056import org.junit.jupiter.api.Tag;
057import org.junit.jupiter.api.Test;
058import org.junit.jupiter.api.TestInfo;
059import org.slf4j.Logger;
060import org.slf4j.LoggerFactory;
061
062import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
063
064/**
065 * Test filters at the HRegion doorstep.
066 */
067@Tag(FilterTests.TAG)
068@Tag(MediumTests.TAG)
069public class TestFilter {
070
071  private final static Logger LOG = LoggerFactory.getLogger(TestFilter.class);
072  private HRegion region;
073  private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
074
075  //
076  // Rows, Qualifiers, and Values are in two groups, One and Two.
077  //
078
079  private static final byte[][] ROWS_ONE = { Bytes.toBytes("testRowOne-0"),
080    Bytes.toBytes("testRowOne-1"), Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") };
081
082  private static final byte[][] ROWS_TWO = { Bytes.toBytes("testRowTwo-0"),
083    Bytes.toBytes("testRowTwo-1"), Bytes.toBytes("testRowTwo-2"), Bytes.toBytes("testRowTwo-3") };
084
085  private static final byte[][] ROWS_THREE =
086    { Bytes.toBytes("testRowThree-0"), Bytes.toBytes("testRowThree-1"),
087      Bytes.toBytes("testRowThree-2"), Bytes.toBytes("testRowThree-3") };
088
089  private static final byte[][] ROWS_FOUR =
090    { Bytes.toBytes("testRowFour-0"), Bytes.toBytes("testRowFour-1"),
091      Bytes.toBytes("testRowFour-2"), Bytes.toBytes("testRowFour-3") };
092
093  private static final byte[][] FAMILIES =
094    { Bytes.toBytes("testFamilyOne"), Bytes.toBytes("testFamilyTwo") };
095
096  private static final byte[][] FAMILIES_1 =
097    { Bytes.toBytes("testFamilyThree"), Bytes.toBytes("testFamilyFour") };
098
099  private static final byte[][] QUALIFIERS_ONE =
100    { Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"),
101      Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3") };
102
103  private static final byte[][] QUALIFIERS_TWO =
104    { Bytes.toBytes("testQualifierTwo-0"), Bytes.toBytes("testQualifierTwo-1"),
105      Bytes.toBytes("testQualifierTwo-2"), Bytes.toBytes("testQualifierTwo-3") };
106
107  private static final byte[][] QUALIFIERS_THREE =
108    { Bytes.toBytes("testQualifierThree-0"), Bytes.toBytes("testQualifierThree-1"),
109      Bytes.toBytes("testQualifierThree-2"), Bytes.toBytes("testQualifierThree-3") };
110
111  private static final byte[][] QUALIFIERS_FOUR =
112    { Bytes.toBytes("testQualifierFour-0"), Bytes.toBytes("testQualifierFour-1"),
113      Bytes.toBytes("testQualifierFour-2"), Bytes.toBytes("testQualifierFour-3") };
114
115  private static final byte[][] QUALIFIERS_FIVE =
116    { Bytes.toBytes("testQualifierFive-0"), Bytes.toBytes("testQualifierFive-1") };
117
118  private static final byte[][] VALUES =
119    { Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo") };
120
121  byte[][] NEW_FAMILIES = { Bytes.toBytes("f1"), Bytes.toBytes("f2") };
122
123  private long numRows = (long) ROWS_ONE.length + ROWS_TWO.length;
124  private long colsPerRow = (long) FAMILIES.length * QUALIFIERS_ONE.length;
125
126  @BeforeEach
127  public void setUp() throws Exception {
128    TableDescriptor tableDescriptor = TableDescriptorBuilder
129      .newBuilder(TableName.valueOf("TestFilter"))
130      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILIES[0]).setMinVersions(100)
131        .setMaxVersions(100).build())
132      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[1]))
133      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[0]))
134      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES_1[1]))
135      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[0]))
136      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(NEW_FAMILIES[1])).build();
137    RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
138    this.region = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
139      TEST_UTIL.getConfiguration(), tableDescriptor);
140
141    // Insert first half
142    for (byte[] ROW : ROWS_ONE) {
143      Put p = new Put(ROW);
144      p.setDurability(Durability.SKIP_WAL);
145      for (byte[] QUALIFIER : QUALIFIERS_ONE) {
146        p.addColumn(FAMILIES[0], QUALIFIER, VALUES[0]);
147      }
148      this.region.put(p);
149    }
150    for (byte[] ROW : ROWS_TWO) {
151      Put p = new Put(ROW);
152      p.setDurability(Durability.SKIP_WAL);
153      for (byte[] QUALIFIER : QUALIFIERS_TWO) {
154        p.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]);
155      }
156      this.region.put(p);
157    }
158
159    // Flush
160    this.region.flush(true);
161
162    // Insert second half (reverse families)
163    for (byte[] ROW : ROWS_ONE) {
164      Put p = new Put(ROW);
165      p.setDurability(Durability.SKIP_WAL);
166      for (byte[] QUALIFIER : QUALIFIERS_ONE) {
167        p.addColumn(FAMILIES[1], QUALIFIER, VALUES[0]);
168      }
169      this.region.put(p);
170    }
171    for (byte[] ROW : ROWS_TWO) {
172      Put p = new Put(ROW);
173      p.setDurability(Durability.SKIP_WAL);
174      for (byte[] QUALIFIER : QUALIFIERS_TWO) {
175        p.addColumn(FAMILIES[0], QUALIFIER, VALUES[1]);
176      }
177      this.region.put(p);
178    }
179
180    // Delete the second qualifier from all rows and families
181    for (byte[] ROW : ROWS_ONE) {
182      Delete d = new Delete(ROW);
183      d.addColumns(FAMILIES[0], QUALIFIERS_ONE[1]);
184      d.addColumns(FAMILIES[1], QUALIFIERS_ONE[1]);
185      this.region.delete(d);
186    }
187    for (byte[] ROW : ROWS_TWO) {
188      Delete d = new Delete(ROW);
189      d.addColumns(FAMILIES[0], QUALIFIERS_TWO[1]);
190      d.addColumns(FAMILIES[1], QUALIFIERS_TWO[1]);
191      this.region.delete(d);
192    }
193    colsPerRow -= 2;
194
195    // Delete the second rows from both groups, one column at a time
196    for (byte[] QUALIFIER : QUALIFIERS_ONE) {
197      Delete d = new Delete(ROWS_ONE[1]);
198      d.addColumns(FAMILIES[0], QUALIFIER);
199      d.addColumns(FAMILIES[1], QUALIFIER);
200      this.region.delete(d);
201    }
202    for (byte[] QUALIFIER : QUALIFIERS_TWO) {
203      Delete d = new Delete(ROWS_TWO[1]);
204      d.addColumns(FAMILIES[0], QUALIFIER);
205      d.addColumns(FAMILIES[1], QUALIFIER);
206      this.region.delete(d);
207    }
208    numRows -= 2;
209  }
210
211  @AfterEach
212  public void tearDown() throws Exception {
213    HBaseTestingUtil.closeRegionAndWAL(region);
214  }
215
216  @Test
217  public void testRegionScannerReseek() throws Exception {
218    // create new rows and column family to show how reseek works..
219    for (byte[] ROW : ROWS_THREE) {
220      Put p = new Put(ROW);
221      p.setDurability(Durability.SKIP_WAL);
222      for (byte[] QUALIFIER : QUALIFIERS_THREE) {
223        p.addColumn(FAMILIES[0], QUALIFIER, VALUES[0]);
224
225      }
226      this.region.put(p);
227    }
228    for (byte[] ROW : ROWS_FOUR) {
229      Put p = new Put(ROW);
230      p.setDurability(Durability.SKIP_WAL);
231      for (byte[] QUALIFIER : QUALIFIERS_FOUR) {
232        p.addColumn(FAMILIES[1], QUALIFIER, VALUES[1]);
233      }
234      this.region.put(p);
235    }
236    // Flush
237    this.region.flush(true);
238
239    // Insert second half (reverse families)
240    for (byte[] ROW : ROWS_THREE) {
241      Put p = new Put(ROW);
242      p.setDurability(Durability.SKIP_WAL);
243      for (byte[] QUALIFIER : QUALIFIERS_THREE) {
244        p.addColumn(FAMILIES[1], QUALIFIER, VALUES[0]);
245      }
246      this.region.put(p);
247    }
248    for (byte[] ROW : ROWS_FOUR) {
249      Put p = new Put(ROW);
250      p.setDurability(Durability.SKIP_WAL);
251      for (byte[] QUALIFIER : QUALIFIERS_FOUR) {
252        p.addColumn(FAMILIES[0], QUALIFIER, VALUES[1]);
253      }
254      this.region.put(p);
255    }
256
257    Scan s = new Scan();
258    // set a start row
259    s.withStartRow(ROWS_FOUR[1]);
260    RegionScanner scanner = region.getScanner(s);
261
262    // reseek to row three.
263    scanner.reseek(ROWS_THREE[1]);
264    List<Cell> results = new ArrayList<>();
265
266    // the results should belong to ROWS_THREE[1]
267    scanner.next(results);
268    for (Cell keyValue : results) {
269      assertTrue(CellUtil.matchingRows(keyValue, ROWS_THREE[1]),
270        "The rows with ROWS_TWO as row key should be appearing.");
271    }
272    // again try to reseek to a value before ROWS_THREE[1]
273    scanner.reseek(ROWS_ONE[1]);
274    results = new ArrayList<>();
275    // This time no seek would have been done to ROWS_ONE[1]
276    scanner.next(results);
277    for (Cell keyValue : results) {
278      assertFalse(Bytes.toString(CellUtil.cloneRow(keyValue)).contains("testRowOne"),
279        "Cannot rewind back to a value less than previous reseek.");
280    }
281  }
282
283  @Test
284  public void testNoFilter() throws Exception {
285    // No filter
286    long expectedRows = this.numRows;
287    long expectedKeys = this.colsPerRow;
288
289    // Both families
290    Scan s = new Scan();
291    verifyScan(s, expectedRows, expectedKeys);
292
293    // One family
294    s = new Scan();
295    s.addFamily(FAMILIES[0]);
296    verifyScan(s, expectedRows, expectedKeys / 2);
297  }
298
299  @Test
300  public void testPrefixFilter() throws Exception {
301    // Grab rows from group one (half of total)
302    long expectedRows = this.numRows / 2;
303    long expectedKeys = this.colsPerRow;
304    Scan s = new Scan();
305    s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
306    verifyScan(s, expectedRows, expectedKeys);
307  }
308
309  @Test
310  public void testPrefixFilterWithReverseScan() throws Exception {
311    // Grab rows from group one (half of total)
312    long expectedRows = this.numRows / 2;
313    long expectedKeys = this.colsPerRow;
314    Scan s = new Scan();
315    s.setReversed(true);
316    s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
317    verifyScan(s, expectedRows, expectedKeys);
318  }
319
320  @Test
321  public void testPageFilter() throws Exception {
322
323    // KVs in first 6 rows
324    KeyValue[] expectedKVs = {
325      // testRowOne-0
326      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
327      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
328      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
329      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
330      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
331      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
332      // testRowOne-2
333      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
334      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
335      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
336      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
337      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
338      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
339      // testRowOne-3
340      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
341      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
342      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
343      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
344      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
345      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
346      // testRowTwo-0
347      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
348      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
349      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
350      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
351      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
352      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
353      // testRowTwo-2
354      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
355      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
356      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
357      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
358      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
359      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
360      // testRowTwo-3
361      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
362      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
363      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
364      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
365      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
366      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) };
367
368    // Grab all 6 rows
369    long expectedRows = 6;
370    long expectedKeys = this.colsPerRow;
371    Scan s = new Scan();
372    s.setFilter(new PageFilter(expectedRows));
373    verifyScan(s, expectedRows, expectedKeys);
374    s.setFilter(new PageFilter(expectedRows));
375    verifyScanFull(s, expectedKVs);
376
377    // Grab first 4 rows (6 cols per row)
378    expectedRows = 4;
379    expectedKeys = this.colsPerRow;
380    s = new Scan();
381    s.setFilter(new PageFilter(expectedRows));
382    verifyScan(s, expectedRows, expectedKeys);
383    s.setFilter(new PageFilter(expectedRows));
384    verifyScanFull(s, Arrays.copyOf(expectedKVs, 24));
385
386    // Grab first 2 rows
387    expectedRows = 2;
388    expectedKeys = this.colsPerRow;
389    s = new Scan();
390    s.setFilter(new PageFilter(expectedRows));
391    verifyScan(s, expectedRows, expectedKeys);
392    s.setFilter(new PageFilter(expectedRows));
393    verifyScanFull(s, Arrays.copyOf(expectedKVs, 12));
394
395    // Grab first row
396    expectedRows = 1;
397    expectedKeys = this.colsPerRow;
398    s = new Scan();
399    s.setFilter(new PageFilter(expectedRows));
400    verifyScan(s, expectedRows, expectedKeys);
401    s.setFilter(new PageFilter(expectedRows));
402    verifyScanFull(s, Arrays.copyOf(expectedKVs, 6));
403
404  }
405
406  @Test
407  public void testPageFilterWithReverseScan() throws Exception {
408    // KVs in first 6 rows
409    KeyValue[] expectedKVs = {
410      // testRowOne-0
411      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
412      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
413      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
414      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
415      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
416      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
417      // testRowOne-2
418      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
419      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
420      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
421      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
422      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
423      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
424      // testRowOne-3
425      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
426      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
427      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
428      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
429      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
430      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
431      // testRowTwo-0
432      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
433      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
434      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
435      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
436      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
437      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
438      // testRowTwo-2
439      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
440      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
441      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
442      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
443      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
444      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
445      // testRowTwo-3
446      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
447      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
448      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
449      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
450      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
451      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) };
452
453    // Grab all 6 rows
454    long expectedRows = 6;
455    long expectedKeys = this.colsPerRow;
456    Scan s = new Scan();
457    s.setReversed(true);
458    s.setFilter(new PageFilter(expectedRows));
459    verifyScan(s, expectedRows, expectedKeys);
460
461    // Grab first 4 rows (6 cols per row)
462    expectedRows = 4;
463    expectedKeys = this.colsPerRow;
464    s = new Scan();
465    s.setReversed(true);
466    s.setFilter(new PageFilter(expectedRows));
467    verifyScan(s, expectedRows, expectedKeys);
468
469    // Grab first 2 rows
470    expectedRows = 2;
471    expectedKeys = this.colsPerRow;
472    s = new Scan();
473    s.setReversed(true);
474    s.setFilter(new PageFilter(expectedRows));
475    verifyScan(s, expectedRows, expectedKeys);
476
477    // Grab first row
478    expectedRows = 1;
479    expectedKeys = this.colsPerRow;
480    s = new Scan();
481    s.setReversed(true);
482    s.setFilter(new PageFilter(expectedRows));
483    verifyScan(s, expectedRows, expectedKeys);
484  }
485
486  @Test
487  public void testWhileMatchFilterWithFilterRowWithReverseScan() throws Exception {
488    final int pageSize = 4;
489
490    Scan s = new Scan();
491    s.setReversed(true);
492    WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize));
493    s.setFilter(filter);
494
495    InternalScanner scanner = this.region.getScanner(s);
496    int scannerCounter = 0;
497    while (true) {
498      boolean isMoreResults = scanner.next(new ArrayList<>());
499      scannerCounter++;
500
501      if (scannerCounter >= pageSize) {
502        assertTrue(filter.filterAllRemaining(),
503          "The WhileMatchFilter should now filter all remaining");
504      }
505      if (!isMoreResults) {
506        break;
507      }
508    }
509    scanner.close();
510    assertEquals(pageSize, scannerCounter, "The page filter returned more rows than expected");
511  }
512
513  @Test
514  public void testWhileMatchFilterWithFilterRowKeyWithReverseScan() throws Exception {
515    Scan s = new Scan();
516    String prefix = "testRowOne";
517    WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix)));
518    s.setFilter(filter);
519    s.setReversed(true);
520
521    InternalScanner scanner = this.region.getScanner(s);
522    while (true) {
523      ArrayList<Cell> values = new ArrayList<>();
524      boolean isMoreResults = scanner.next(values);
525      if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) {
526        assertTrue(filter.filterAllRemaining(),
527          "The WhileMatchFilter should now filter all remaining");
528      }
529      if (!isMoreResults) {
530        break;
531      }
532    }
533    scanner.close();
534  }
535
536  /**
537   * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the
538   * {@link Filter#filterRow()} method. See HBASE-2258.
539   */
540  @Test
541  public void testWhileMatchFilterWithFilterRow() throws Exception {
542    final int pageSize = 4;
543
544    Scan s = new Scan();
545    WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize));
546    s.setFilter(filter);
547
548    InternalScanner scanner = this.region.getScanner(s);
549    int scannerCounter = 0;
550    while (true) {
551      boolean isMoreResults = scanner.next(new ArrayList<>());
552      scannerCounter++;
553
554      if (scannerCounter >= pageSize) {
555        assertTrue(filter.filterAllRemaining(),
556          "The WhileMatchFilter should now filter all remaining");
557      }
558      if (!isMoreResults) {
559        break;
560      }
561    }
562    assertEquals(pageSize, scannerCounter, "The page filter returned more rows than expected");
563  }
564
565  /**
566   * The following filter simulates a pre-0.96 filter where filterRow() is defined while
567   * hasFilterRow() returns false
568   */
569  static class OldTestFilter extends FilterBase {
570    @Override
571    public byte[] toByteArray() {
572      return null;
573    }
574
575    @Override
576    public boolean hasFilterRow() {
577      return false;
578    }
579
580    @Override
581    public boolean filterRow() {
582      // always filter out rows
583      return true;
584    }
585
586    @Override
587    public ReturnCode filterCell(final Cell ignored) throws IOException {
588      return ReturnCode.INCLUDE;
589    }
590  }
591
592  /**
593   * The following test is to ensure old(such as hbase0.94) filterRow() can be correctly fired in
594   * 0.96+ code base. See HBASE-10366
595   */
596  @Test
597  public void test94FilterRowCompatibility() throws Exception {
598    Scan s = new Scan();
599    OldTestFilter filter = new OldTestFilter();
600    s.setFilter(filter);
601
602    InternalScanner scanner = this.region.getScanner(s);
603    ArrayList<Cell> values = new ArrayList<>();
604    scanner.next(values);
605    assertTrue(values.isEmpty(), "All rows should be filtered out");
606  }
607
608  /**
609   * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the
610   * {@link Filter#filterRowKey(Cell)} method. See HBASE-2258.
611   */
612  @Test
613  public void testWhileMatchFilterWithFilterRowKey() throws Exception {
614    Scan s = new Scan();
615    String prefix = "testRowOne";
616    WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix)));
617    s.setFilter(filter);
618
619    InternalScanner scanner = this.region.getScanner(s);
620    while (true) {
621      ArrayList<Cell> values = new ArrayList<>();
622      boolean isMoreResults = scanner.next(values);
623      if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) {
624        assertTrue(filter.filterAllRemaining(),
625          "The WhileMatchFilter should now filter all remaining");
626      }
627      if (!isMoreResults) {
628        break;
629      }
630    }
631  }
632
633  /**
634   * Tests the the {@link WhileMatchFilter} works in combination with a {@link Filter} that uses the
635   * {@link Filter#filterCell(Cell)} method. See HBASE-2258.
636   */
637  @Test
638  public void testWhileMatchFilterWithFilterCell() throws Exception {
639    Scan s = new Scan();
640    WhileMatchFilter filter = new WhileMatchFilter(new SingleColumnValueFilter(FAMILIES[0],
641      QUALIFIERS_ONE[0], CompareOperator.EQUAL, Bytes.toBytes("foo")));
642    s.setFilter(filter);
643
644    InternalScanner scanner = this.region.getScanner(s);
645    while (true) {
646      ArrayList<Cell> values = new ArrayList<>();
647      boolean isMoreResults = scanner.next(values);
648      assertTrue(filter.filterAllRemaining(),
649        "The WhileMatchFilter should now filter all remaining");
650      if (!isMoreResults) {
651        break;
652      }
653    }
654  }
655
656  @Test
657  public void testInclusiveStopFilter() throws IOException {
658
659    // Grab rows from group one
660
661    // If we just use start/stop row, we get total/2 - 1 rows
662    long expectedRows = (this.numRows / 2) - 1;
663    long expectedKeys = this.colsPerRow;
664    Scan s = new Scan().withStartRow(Bytes.toBytes("testRowOne-0"))
665      .withStopRow(Bytes.toBytes("testRowOne-3"));
666    verifyScan(s, expectedRows, expectedKeys);
667
668    // Now use start row with inclusive stop filter
669    expectedRows = this.numRows / 2;
670    s = new Scan().withStartRow(Bytes.toBytes("testRowOne-0"));
671    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-3")));
672    verifyScan(s, expectedRows, expectedKeys);
673
674    // Grab rows from group two
675
676    // If we just use start/stop row, we get total/2 - 1 rows
677    expectedRows = (this.numRows / 2) - 1;
678    expectedKeys = this.colsPerRow;
679    s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-0"))
680      .withStopRow(Bytes.toBytes("testRowTwo-3"));
681    verifyScan(s, expectedRows, expectedKeys);
682
683    // Now use start row with inclusive stop filter
684    expectedRows = this.numRows / 2;
685    s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-0"));
686    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-3")));
687    verifyScan(s, expectedRows, expectedKeys);
688
689  }
690
691  @Test
692  public void testInclusiveStopFilterWithReverseScan() throws IOException {
693
694    // Grab rows from group one
695
696    // If we just use start/stop row, we get total/2 - 1 rows
697    long expectedRows = (this.numRows / 2) - 1;
698    long expectedKeys = this.colsPerRow;
699    Scan s = new Scan().withStartRow(Bytes.toBytes("testRowOne-3"))
700      .withStopRow(Bytes.toBytes("testRowOne-0"));
701    s.setReversed(true);
702    verifyScan(s, expectedRows, expectedKeys);
703
704    // Now use start row with inclusive stop filter
705    expectedRows = this.numRows / 2;
706    s = new Scan().withStartRow(Bytes.toBytes("testRowOne-3"));
707    s.setReversed(true);
708    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-0")));
709    verifyScan(s, expectedRows, expectedKeys);
710
711    // Grab rows from group two
712
713    // If we just use start/stop row, we get total/2 - 1 rows
714    expectedRows = (this.numRows / 2) - 1;
715    expectedKeys = this.colsPerRow;
716    s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-3"))
717      .withStopRow(Bytes.toBytes("testRowTwo-0"));
718    s.setReversed(true);
719    verifyScan(s, expectedRows, expectedKeys);
720
721    // Now use start row with inclusive stop filter
722    expectedRows = this.numRows / 2;
723    s = new Scan().withStartRow(Bytes.toBytes("testRowTwo-3"));
724    s.setReversed(true);
725    s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-0")));
726    verifyScan(s, expectedRows, expectedKeys);
727
728  }
729
730  @Test
731  public void testQualifierFilter() throws IOException {
732
733    // Match two keys (one from each family) in half the rows
734    long expectedRows = this.numRows / 2;
735    long expectedKeys = 2;
736    Filter f = new QualifierFilter(CompareOperator.EQUAL,
737      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
738    Scan s = new Scan();
739    s.setFilter(f);
740    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
741
742    // Match keys less than same qualifier
743    // Expect only two keys (one from each family) in half the rows
744    expectedRows = this.numRows / 2;
745    expectedKeys = 2;
746    f = new QualifierFilter(CompareOperator.LESS,
747      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
748    s = new Scan();
749    s.setFilter(f);
750    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
751
752    // Match keys less than or equal
753    // Expect four keys (two from each family) in half the rows
754    expectedRows = this.numRows / 2;
755    expectedKeys = 4;
756    f = new QualifierFilter(CompareOperator.LESS_OR_EQUAL,
757      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
758    s = new Scan();
759    s.setFilter(f);
760    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
761
762    // Match keys not equal
763    // Expect four keys (two from each family)
764    // Only look in first group of rows
765    expectedRows = this.numRows / 2;
766    expectedKeys = 4;
767    f = new QualifierFilter(CompareOperator.NOT_EQUAL,
768      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
769    s =
770      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
771    s.setFilter(f);
772    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
773
774    // Match keys greater or equal
775    // Expect four keys (two from each family)
776    // Only look in first group of rows
777    expectedRows = this.numRows / 2;
778    expectedKeys = 4;
779    f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL,
780      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
781    s =
782      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
783    s.setFilter(f);
784    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
785
786    // Match keys greater
787    // Expect two keys (one from each family)
788    // Only look in first group of rows
789    expectedRows = this.numRows / 2;
790    expectedKeys = 2;
791    f = new QualifierFilter(CompareOperator.GREATER,
792      new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
793    s =
794      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
795    s.setFilter(f);
796    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
797
798    // Match keys not equal to
799    // Look across rows and fully validate the keys and ordering
800    // Expect varied numbers of keys, 4 per row in group one, 6 per row in group two
801    f = new QualifierFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(QUALIFIERS_ONE[2]));
802    s = new Scan();
803    s.setFilter(f);
804
805    KeyValue[] kvs = {
806      // testRowOne-0
807      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
808      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
809      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
810      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
811      // testRowOne-2
812      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
813      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
814      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
815      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
816      // testRowOne-3
817      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
818      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
819      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
820      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
821      // testRowTwo-0
822      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
823      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
824      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
825      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
826      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
827      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
828      // testRowTwo-2
829      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
830      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
831      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
832      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
833      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
834      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
835      // testRowTwo-3
836      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
837      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
838      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
839      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
840      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
841      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
842    verifyScanFull(s, kvs);
843
844    // Test across rows and groups with a regex
845    // Filter out "test*-2"
846    // Expect 4 keys per row across both groups
847    f = new QualifierFilter(CompareOperator.NOT_EQUAL, new RegexStringComparator("test.+-2"));
848    s = new Scan();
849    s.setFilter(f);
850
851    kvs = new KeyValue[] {
852      // testRowOne-0
853      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
854      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
855      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
856      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
857      // testRowOne-2
858      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
859      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
860      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
861      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
862      // testRowOne-3
863      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
864      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
865      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
866      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
867      // testRowTwo-0
868      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
869      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
870      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
871      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
872      // testRowTwo-2
873      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
874      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
875      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
876      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
877      // testRowTwo-3
878      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
879      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
880      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
881      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
882    verifyScanFull(s, kvs);
883
884  }
885
886  @Test
887  public void testFamilyFilter() throws IOException {
888    // Match family, only half of columns returned.
889    long expectedRows = this.numRows;
890    long expectedKeys = this.colsPerRow / 2;
891    Filter f =
892      new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testFamilyOne")));
893    Scan s = new Scan();
894    s.setFilter(f);
895    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
896
897    // Match keys less than given family, should return nothing
898    expectedRows = 0;
899    expectedKeys = 0;
900    f = new FamilyFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testFamily")));
901    s = new Scan();
902    s.setFilter(f);
903    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
904
905    // Match keys less than or equal, should return half of columns
906    expectedRows = this.numRows;
907    expectedKeys = this.colsPerRow / 2;
908    f = new FamilyFilter(CompareOperator.LESS_OR_EQUAL,
909      new BinaryComparator(Bytes.toBytes("testFamilyOne")));
910    s = new Scan();
911    s.setFilter(f);
912    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
913
914    // Match keys from second family
915    // look only in second group of rows
916    expectedRows = this.numRows / 2;
917    expectedKeys = this.colsPerRow / 2;
918    f = new FamilyFilter(CompareOperator.NOT_EQUAL,
919      new BinaryComparator(Bytes.toBytes("testFamilyOne")));
920    s =
921      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
922    s.setFilter(f);
923    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
924
925    // Match all columns
926    // look only in second group of rows
927    expectedRows = this.numRows / 2;
928    expectedKeys = this.colsPerRow;
929    f = new FamilyFilter(CompareOperator.GREATER_OR_EQUAL,
930      new BinaryComparator(Bytes.toBytes("testFamilyOne")));
931    s =
932      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
933    s.setFilter(f);
934    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
935
936    // Match all columns in second family
937    // look only in second group of rows
938    expectedRows = this.numRows / 2;
939    expectedKeys = this.colsPerRow / 2;
940    f = new FamilyFilter(CompareOperator.GREATER,
941      new BinaryComparator(Bytes.toBytes("testFamilyOne")));
942    s =
943      new Scan().withStartRow(HConstants.EMPTY_START_ROW).withStopRow(Bytes.toBytes("testRowTwo"));
944    s.setFilter(f);
945    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
946
947    // Match keys not equal to given family
948    // Look across rows and fully validate the keys and ordering
949    f = new FamilyFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(FAMILIES[1]));
950    s = new Scan();
951    s.setFilter(f);
952
953    KeyValue[] kvs = {
954      // testRowOne-0
955      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
956      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
957      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
958      // testRowOne-2
959      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
960      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
961      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
962      // testRowOne-3
963      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
964      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
965      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
966      // testRowTwo-0
967      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
968      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
969      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
970      // testRowTwo-2
971      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
972      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
973      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
974      // testRowTwo-3
975      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
976      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
977      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), };
978    verifyScanFull(s, kvs);
979
980    // Test across rows and groups with a regex
981    // Filter out "test*-2"
982    // Expect 4 keys per row across both groups
983    f = new FamilyFilter(CompareOperator.NOT_EQUAL, new RegexStringComparator("test.*One"));
984    s = new Scan();
985    s.setFilter(f);
986
987    kvs = new KeyValue[] {
988      // testRowOne-0
989      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
990      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
991      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
992      // testRowOne-2
993      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
994      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
995      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
996      // testRowOne-3
997      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
998      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
999      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1000      // testRowTwo-0
1001      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1002      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1003      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1004      // testRowTwo-2
1005      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1006      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1007      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1008      // testRowTwo-3
1009      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1010      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1011      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
1012    verifyScanFull(s, kvs);
1013  }
1014
1015  @Test
1016  public void testRowFilter() throws IOException {
1017
1018    // Match a single row, all keys
1019    long expectedRows = 1;
1020    long expectedKeys = this.colsPerRow;
1021    Filter f =
1022      new RowFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1023    Scan s = new Scan();
1024    s.setFilter(f);
1025    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1026
1027    // Match a two rows, one from each group, using regex
1028    expectedRows = 2;
1029    expectedKeys = this.colsPerRow;
1030    f = new RowFilter(CompareOperator.EQUAL, new RegexStringComparator("testRow.+-2"));
1031    s = new Scan();
1032    s.setFilter(f);
1033    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1034
1035    // Match rows less than
1036    // Expect all keys in one row
1037    expectedRows = 1;
1038    expectedKeys = this.colsPerRow;
1039    f = new RowFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1040    s = new Scan();
1041    s.setFilter(f);
1042    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1043
1044    // Match rows less than or equal
1045    // Expect all keys in two rows
1046    expectedRows = 2;
1047    expectedKeys = this.colsPerRow;
1048    f = new RowFilter(CompareOperator.LESS_OR_EQUAL,
1049      new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1050    s = new Scan();
1051    s.setFilter(f);
1052    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1053
1054    // Match rows not equal
1055    // Expect all keys in all but one row
1056    expectedRows = this.numRows - 1;
1057    expectedKeys = this.colsPerRow;
1058    f =
1059      new RowFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1060    s = new Scan();
1061    s.setFilter(f);
1062    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1063
1064    // Match keys greater or equal
1065    // Expect all keys in all but one row
1066    expectedRows = this.numRows - 1;
1067    expectedKeys = this.colsPerRow;
1068    f = new RowFilter(CompareOperator.GREATER_OR_EQUAL,
1069      new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1070    s = new Scan();
1071    s.setFilter(f);
1072    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1073
1074    // Match keys greater
1075    // Expect all keys in all but two rows
1076    expectedRows = this.numRows - 2;
1077    expectedKeys = this.colsPerRow;
1078    f = new RowFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1079    s = new Scan();
1080    s.setFilter(f);
1081    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1082
1083    // Match rows not equal to testRowTwo-2
1084    // Look across rows and fully validate the keys and ordering
1085    // Should see all keys in all rows but testRowTwo-2
1086    f =
1087      new RowFilter(CompareOperator.NOT_EQUAL, new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1088    s = new Scan();
1089    s.setFilter(f);
1090
1091    KeyValue[] kvs = {
1092      // testRowOne-0
1093      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1094      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1095      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1096      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1097      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1098      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1099      // testRowOne-3
1100      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1101      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1102      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1103      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1104      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1105      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1106      // testRowTwo-0
1107      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1108      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1109      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1110      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1111      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1112      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1113      // testRowTwo-2
1114      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1115      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1116      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1117      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1118      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1119      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1120      // testRowTwo-3
1121      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1122      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1123      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1124      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1125      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1126      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
1127    verifyScanFull(s, kvs);
1128
1129    // Test across rows and groups with a regex
1130    // Filter out everything that doesn't match "*-2"
1131    // Expect all keys in two rows
1132    f = new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2"));
1133    s = new Scan();
1134    s.setFilter(f);
1135
1136    kvs = new KeyValue[] {
1137      // testRowOne-2
1138      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1139      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1140      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1141      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1142      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1143      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1144      // testRowTwo-2
1145      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1146      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1147      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1148      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1149      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1150      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) };
1151    verifyScanFull(s, kvs);
1152
1153  }
1154
1155  @Test
1156  public void testValueFilter() throws IOException {
1157
1158    // Match group one rows
1159    long expectedRows = this.numRows / 2;
1160    long expectedKeys = this.colsPerRow;
1161    Filter f =
1162      new ValueFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testValueOne")));
1163    Scan s = new Scan();
1164    s.setFilter(f);
1165    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1166
1167    // Match group two rows
1168    expectedRows = this.numRows / 2;
1169    expectedKeys = this.colsPerRow;
1170    f = new ValueFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("testValueTwo")));
1171    s = new Scan();
1172    s.setFilter(f);
1173    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1174
1175    // Match all values using regex
1176    expectedRows = this.numRows;
1177    expectedKeys = this.colsPerRow;
1178    f = new ValueFilter(CompareOperator.EQUAL, new RegexStringComparator("testValue((One)|(Two))"));
1179    s = new Scan();
1180    s.setFilter(f);
1181    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1182
1183    // Match values less than
1184    // Expect group one rows
1185    expectedRows = this.numRows / 2;
1186    expectedKeys = this.colsPerRow;
1187    f = new ValueFilter(CompareOperator.LESS, new BinaryComparator(Bytes.toBytes("testValueTwo")));
1188    s = new Scan();
1189    s.setFilter(f);
1190    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1191
1192    // Match values less than or equal
1193    // Expect all rows
1194    expectedRows = this.numRows;
1195    expectedKeys = this.colsPerRow;
1196    f = new ValueFilter(CompareOperator.LESS_OR_EQUAL,
1197      new BinaryComparator(Bytes.toBytes("testValueTwo")));
1198    s = new Scan();
1199    s.setFilter(f);
1200    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1201
1202    // Match values less than or equal
1203    // Expect group one rows
1204    expectedRows = this.numRows / 2;
1205    expectedKeys = this.colsPerRow;
1206    f = new ValueFilter(CompareOperator.LESS_OR_EQUAL,
1207      new BinaryComparator(Bytes.toBytes("testValueOne")));
1208    s = new Scan();
1209    s.setFilter(f);
1210    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1211
1212    // Match values not equal
1213    // Expect half the rows
1214    expectedRows = this.numRows / 2;
1215    expectedKeys = this.colsPerRow;
1216    f = new ValueFilter(CompareOperator.NOT_EQUAL,
1217      new BinaryComparator(Bytes.toBytes("testValueOne")));
1218    s = new Scan();
1219    s.setFilter(f);
1220    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1221
1222    // Match values greater or equal
1223    // Expect all rows
1224    expectedRows = this.numRows;
1225    expectedKeys = this.colsPerRow;
1226    f = new ValueFilter(CompareOperator.GREATER_OR_EQUAL,
1227      new BinaryComparator(Bytes.toBytes("testValueOne")));
1228    s = new Scan();
1229    s.setFilter(f);
1230    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1231
1232    // Match values greater
1233    // Expect half rows
1234    expectedRows = this.numRows / 2;
1235    expectedKeys = this.colsPerRow;
1236    f =
1237      new ValueFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("testValueOne")));
1238    s = new Scan();
1239    s.setFilter(f);
1240    verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1241
1242    // Match values not equal to testValueOne
1243    // Look across rows and fully validate the keys and ordering
1244    // Should see all keys in all group two rows
1245    f = new ValueFilter(CompareOperator.NOT_EQUAL,
1246      new BinaryComparator(Bytes.toBytes("testValueOne")));
1247    s = new Scan();
1248    s.setFilter(f);
1249
1250    KeyValue[] kvs = {
1251      // testRowTwo-0
1252      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1253      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1254      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1255      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1256      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1257      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1258      // testRowTwo-2
1259      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1260      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1261      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1262      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1263      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1264      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1265      // testRowTwo-3
1266      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1267      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1268      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1269      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1270      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1271      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
1272    verifyScanFull(s, kvs);
1273  }
1274
1275  @Test
1276  public void testSkipFilter() throws IOException {
1277
1278    // Test for qualifier regex: "testQualifierOne-2"
1279    // Should only get rows from second group, and all keys
1280    Filter f = new SkipFilter(new QualifierFilter(CompareOperator.NOT_EQUAL,
1281      new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))));
1282    Scan s = new Scan();
1283    s.setFilter(f);
1284
1285    KeyValue[] kvs = {
1286      // testRowTwo-0
1287      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1288      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1289      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1290      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1291      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1292      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1293      // testRowTwo-2
1294      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1295      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1296      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1297      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1298      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1299      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1300      // testRowTwo-3
1301      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1302      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1303      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1304      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1305      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1306      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
1307    verifyScanFull(s, kvs);
1308  }
1309
1310  // TODO: This is important... need many more tests for ordering, etc
1311  // There are limited tests elsewhere but we need HRegion level ones here
1312  @Test
1313  public void testFilterList() throws IOException {
1314
1315    // Test getting a single row, single key using Row, Qualifier, and Value
1316    // regular expression and substring filters
1317    // Use must pass all
1318    List<Filter> filters = new ArrayList<>();
1319    filters.add(new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2")));
1320    filters.add(new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2")));
1321    filters.add(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("One")));
1322    Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1323    Scan s = new Scan();
1324    s.addFamily(FAMILIES[0]);
1325    s.setFilter(f);
1326    KeyValue[] kvs = { new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]) };
1327    verifyScanFull(s, kvs);
1328
1329    // Test getting everything with a MUST_PASS_ONE filter including row, qf, val
1330    // regular expression and substring filters
1331    filters.clear();
1332    filters.add(new RowFilter(CompareOperator.EQUAL, new RegexStringComparator(".+Two.+")));
1333    filters.add(new QualifierFilter(CompareOperator.EQUAL, new RegexStringComparator(".+-2")));
1334    filters.add(new ValueFilter(CompareOperator.EQUAL, new SubstringComparator("One")));
1335    f = new FilterList(Operator.MUST_PASS_ONE, filters);
1336    s = new Scan();
1337    s.setFilter(f);
1338    verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow);
1339
1340  }
1341
1342  @Test
1343  public void testFirstKeyOnlyFilter() throws IOException {
1344    Scan s = new Scan();
1345    s.setFilter(new FirstKeyOnlyFilter());
1346    // Expected KVs, the first KV from each of the remaining 6 rows
1347    KeyValue[] kvs = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1348      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1349      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1350      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1351      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1352      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]) };
1353    verifyScanFull(s, kvs);
1354  }
1355
1356  @Test
1357  public void testFilterListWithSingleColumnValueFilter() throws IOException {
1358    // Test for HBASE-3191
1359
1360    // Scan using SingleColumnValueFilter
1361    SingleColumnValueFilter f1 =
1362      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]);
1363    f1.setFilterIfMissing(true);
1364    Scan s1 = new Scan();
1365    s1.addFamily(FAMILIES[0]);
1366    s1.setFilter(f1);
1367    KeyValue[] kvs1 = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1368      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1369      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1370      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1371      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1372      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1373      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1374      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1375      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]), };
1376    verifyScanNoEarlyOut(s1, 3, 3);
1377    verifyScanFull(s1, kvs1);
1378
1379    // Scan using another SingleColumnValueFilter, expect disjoint result
1380    SingleColumnValueFilter f2 =
1381      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_TWO[0], CompareOperator.EQUAL, VALUES[1]);
1382    f2.setFilterIfMissing(true);
1383    Scan s2 = new Scan();
1384    s2.addFamily(FAMILIES[0]);
1385    s2.setFilter(f2);
1386    KeyValue[] kvs2 = { new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1387      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1388      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1389      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1390      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1391      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1392      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1393      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1394      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), };
1395    verifyScanNoEarlyOut(s2, 3, 3);
1396    verifyScanFull(s2, kvs2);
1397
1398    // Scan, ORing the two previous filters, expect unified result
1399    FilterList f = new FilterList(Operator.MUST_PASS_ONE);
1400    f.addFilter(f1);
1401    f.addFilter(f2);
1402    Scan s = new Scan();
1403    s.addFamily(FAMILIES[0]);
1404    s.setFilter(f);
1405    KeyValue[] kvs = { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1406      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1407      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1408      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1409      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1410      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1411      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1412      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1413      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1414      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1415      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1416      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1417      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1418      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1419      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1420      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1421      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1422      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), };
1423    verifyScanNoEarlyOut(s, 6, 3);
1424    verifyScanFull(s, kvs);
1425  }
1426
1427  // HBASE-9747
1428  @Test
1429  public void testFilterListWithPrefixFilter(TestInfo testInfo) throws IOException {
1430    byte[] family = Bytes.toBytes("f1");
1431    byte[] qualifier = Bytes.toBytes("q1");
1432    TableDescriptor tableDescriptor =
1433      TableDescriptorBuilder.newBuilder(TableName.valueOf(testInfo.getTestMethod().get().getName()))
1434        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
1435    RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
1436    HRegion testRegion = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
1437      TEST_UTIL.getConfiguration(), tableDescriptor);
1438
1439    for (int i = 0; i < 5; i++) {
1440      Put p = new Put(Bytes.toBytes((char) ('a' + i) + "row"));
1441      p.setDurability(Durability.SKIP_WAL);
1442      p.addColumn(family, qualifier, Bytes.toBytes(String.valueOf(111 + i)));
1443      testRegion.put(p);
1444    }
1445    testRegion.flush(true);
1446
1447    // rows starting with "b"
1448    PrefixFilter pf = new PrefixFilter(new byte[] { 'b' });
1449    // rows with value of column 'q1' set to '113'
1450    SingleColumnValueFilter scvf =
1451      new SingleColumnValueFilter(family, qualifier, CompareOperator.EQUAL, Bytes.toBytes("113"));
1452    // combine these two with OR in a FilterList
1453    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, pf, scvf);
1454
1455    Scan s1 = new Scan();
1456    s1.setFilter(filterList);
1457    InternalScanner scanner = testRegion.getScanner(s1);
1458    List<Cell> results = new ArrayList<>();
1459    int resultCount = 0;
1460    while (scanner.next(results)) {
1461      resultCount++;
1462      byte[] row = CellUtil.cloneRow(results.get(0));
1463      LOG.debug("Found row: " + Bytes.toStringBinary(row));
1464      assertTrue(
1465        Bytes.equals(row, Bytes.toBytes("brow")) || Bytes.equals(row, Bytes.toBytes("crow")));
1466      results.clear();
1467    }
1468    assertEquals(2, resultCount);
1469    scanner.close();
1470
1471    WAL wal = ((HRegion) testRegion).getWAL();
1472    ((HRegion) testRegion).close();
1473    wal.close();
1474  }
1475
1476  @Test
1477  public void testSingleColumnValueFilter() throws IOException {
1478
1479    // From HBASE-1821
1480    // Desired action is to combine two SCVF in a FilterList
1481    // Want to return only rows that match both conditions
1482
1483    // Need to change one of the group one columns to use group two value
1484    Put p = new Put(ROWS_ONE[2]);
1485    p.addColumn(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]);
1486    this.region.put(p);
1487
1488    // Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1])
1489    // Since group two rows don't have these qualifiers, they will pass
1490    // so limiting scan to group one
1491    List<Filter> filters = new ArrayList<>();
1492    filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL,
1493      VALUES[0]));
1494    filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2], CompareOperator.EQUAL,
1495      VALUES[1]));
1496    Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1497    Scan s = new Scan().withStartRow(ROWS_ONE[0]).withStopRow(ROWS_TWO[0]);
1498    s.addFamily(FAMILIES[0]);
1499    s.setFilter(f);
1500    // Expect only one row, all qualifiers
1501    KeyValue[] kvs = { new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1502      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]),
1503      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]) };
1504    verifyScanNoEarlyOut(s, 1, 3);
1505    verifyScanFull(s, kvs);
1506
1507    // In order to get expected behavior without limiting to group one
1508    // need to wrap SCVFs in SkipFilters
1509    filters = new ArrayList<>();
1510    filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1511      CompareOperator.EQUAL, VALUES[0])));
1512    filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
1513      CompareOperator.EQUAL, VALUES[1])));
1514    f = new FilterList(Operator.MUST_PASS_ALL, filters);
1515    s = new Scan().withStartRow(ROWS_ONE[0]).withStopRow(ROWS_TWO[0]);
1516    s.addFamily(FAMILIES[0]);
1517    s.setFilter(f);
1518    // Expect same KVs
1519    verifyScanNoEarlyOut(s, 1, 3);
1520    verifyScanFull(s, kvs);
1521
1522    // More tests from HBASE-1821 for Clint and filterIfMissing flag
1523
1524    byte[][] ROWS_THREE = { Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"),
1525      Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3") };
1526
1527    // Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1])
1528    // Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1])
1529
1530    KeyValue[] srcKVs =
1531      new KeyValue[] { new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1532        new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
1533        new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]),
1534        new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1]) };
1535
1536    for (KeyValue kv : srcKVs) {
1537      Put put = new Put(CellUtil.cloneRow(kv)).add(kv);
1538      put.setDurability(Durability.SKIP_WAL);
1539      this.region.put(put);
1540    }
1541
1542    // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false
1543    // Expect 3 rows (0, 2, 3)
1544    SingleColumnValueFilter scvf =
1545      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]);
1546    s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4"));
1547    s.addFamily(FAMILIES[0]);
1548    s.setFilter(scvf);
1549    kvs = new KeyValue[] { srcKVs[0], srcKVs[2], srcKVs[3] };
1550    verifyScanFull(s, kvs);
1551
1552    // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true
1553    // Expect 1 row (0)
1554    scvf =
1555      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOperator.EQUAL, VALUES[0]);
1556    scvf.setFilterIfMissing(true);
1557    s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4"));
1558    s.addFamily(FAMILIES[0]);
1559    s.setFilter(scvf);
1560    kvs = new KeyValue[] { srcKVs[0] };
1561    verifyScanFull(s, kvs);
1562
1563    // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1564    // Expect 1 row (3)
1565    scvf =
1566      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[1], CompareOperator.EQUAL, VALUES[1]);
1567    scvf.setFilterIfMissing(true);
1568    s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4"));
1569    s.addFamily(FAMILIES[0]);
1570    s.setFilter(scvf);
1571    kvs = new KeyValue[] { srcKVs[3] };
1572    verifyScanFull(s, kvs);
1573
1574    // Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0]
1575    KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]);
1576    this.region.put(new Put(CellUtil.cloneRow(kvA)).add(kvA));
1577
1578    // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1579    // Expect 1 row (3)
1580    scvf =
1581      new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[1], CompareOperator.EQUAL, VALUES[1]);
1582    scvf.setFilterIfMissing(true);
1583    s = new Scan().withStartRow(ROWS_THREE[0]).withStopRow(Bytes.toBytes("rowThree-4"));
1584    s.addFamily(FAMILIES[0]);
1585    s.setFilter(scvf);
1586    kvs = new KeyValue[] { srcKVs[3] };
1587    verifyScanFull(s, kvs);
1588
1589  }
1590
1591  @Test
1592  public void testColumnValueFilter() throws Exception {
1593    // Prepare test rows:
1594    for (int i = 0; i < 2; i++) {
1595      for (int j = 0; j < ROWS_ONE.length; j++) {
1596        Put p1 = new Put(ROWS_ONE[j]).setDurability(Durability.SKIP_WAL);
1597        Put p2 = new Put(ROWS_TWO[j]).setDurability(Durability.SKIP_WAL);
1598        for (byte[] q5 : QUALIFIERS_FIVE) {
1599          p1.addColumn(FAMILIES[0], q5, VALUES[0 + i]).addColumn(FAMILIES[1], q5, VALUES[0 + i]);
1600          p2.addColumn(FAMILIES[0], q5, VALUES[1 - i]).addColumn(FAMILIES[1], q5, VALUES[1 - i]);
1601        }
1602        this.region.put(p1);
1603        this.region.put(p2);
1604      }
1605      this.region.flush(true);
1606    }
1607    // 1. Test = f[0]:q5[0]:v[1]
1608    Scan scan = new Scan().setFilter(
1609      new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.EQUAL, VALUES[1]));
1610    KeyValue[] expectedEquals =
1611      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1612        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1613        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1614        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1615        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1616        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1617        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1618        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
1619    verifyScanFull(scan, expectedEquals);
1620    // 2. Test > f[0]:q5[0]:v[0]
1621    scan.setFilter(
1622      new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0], CompareOperator.GREATER, VALUES[0]));
1623    KeyValue[] expectedGreater =
1624      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1625        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1626        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1627        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1628        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1629        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1630        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1631        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
1632    verifyScanFull(scan, expectedGreater);
1633    // 3. Test >= f[0]:q5[0]:v[0]
1634    // also test readAllVersions(), since FAMILIES[0] allow multiple versions.
1635    scan.readAllVersions().setFilter(new ColumnValueFilter(FAMILIES[0], QUALIFIERS_FIVE[0],
1636      CompareOperator.GREATER_OR_EQUAL, VALUES[0]));
1637    KeyValue[] expectedGreaterOrEqual =
1638      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1639        new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1640        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1641        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1642        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1643        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1644        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1645        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1646        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1647        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1648        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1649        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1650        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1651        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1652        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1653        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]) };
1654    verifyScanFull(scan, expectedGreaterOrEqual);
1655    // 4. Test < f[1]:q5[1]:v[1], FAMILIES[1] doesn't support multiple versions
1656    scan.readVersions(1).setFilter(
1657      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.LESS, VALUES[1]));
1658    KeyValue[] expectedLess =
1659      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1660        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1661        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1662        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
1663    verifyScanFull(scan, expectedLess);
1664    // 5. Test <= f[1]:q5[0]:v[1]
1665    scan.setFilter(new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1],
1666      CompareOperator.LESS_OR_EQUAL, VALUES[1]));
1667    KeyValue[] expectedLessOrEqual =
1668      { new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
1669        new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
1670        new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
1671        new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[1]),
1672        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1673        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1674        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1675        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
1676    verifyScanFull(scan, expectedLessOrEqual);
1677    // 6. Test != f[1]:q5[1]:v[1]
1678    scan.setFilter(
1679      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
1680    KeyValue[] expectedNotEqual =
1681      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1682        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1683        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1684        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
1685    verifyScanFull(scan, expectedNotEqual);
1686    // 7. Test FilterList(MUST_PASS_ONE) combining ColumnValueFilter and QualifierFilter
1687    // (ColumnValueFilter, != f[1]:q5[1]:v[1]) || (QualifierFilter, = q5[0])
1688    List<Filter> orFilters = new ArrayList<>(2);
1689    orFilters.add(
1690      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
1691    orFilters
1692      .add(new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(QUALIFIERS_FIVE[0])));
1693    scan.setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
1694    KeyValue[] expectedMustPassOne =
1695      { new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1696        new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
1697        new KeyValue(ROWS_ONE[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1698        new KeyValue(ROWS_ONE[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
1699        new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1700        new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
1701        new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[1]),
1702        new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[1]),
1703        new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1704        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
1705        new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
1706        new KeyValue(ROWS_TWO[1], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1707        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
1708        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
1709        new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1710        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
1711        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]), // this pass scvf
1712        new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_FIVE[0], VALUES[0]),
1713        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[0], VALUES[0]),
1714        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) }; // this pass scvf
1715    verifyScanFull(scan, expectedMustPassOne);
1716    // 8. Test FilterList(MUST_PASS_ALL) combining ColumnValueFilter and RowFilter
1717    // (ColumnValueFilter, != f[1]:q5[1]:v[1]) && (RowFilter, = prefix:"testRow")
1718    List<Filter> andFilters = new ArrayList<>(2);
1719    andFilters.add(
1720      new ColumnValueFilter(FAMILIES[1], QUALIFIERS_FIVE[1], CompareOperator.NOT_EQUAL, VALUES[1]));
1721    andFilters.add(
1722      new RowFilter(CompareOperator.EQUAL, new BinaryPrefixComparator(Bytes.toBytes("testRow"))));
1723    scan.setFilter(new FilterList(Operator.MUST_PASS_ALL, andFilters));
1724    KeyValue[] expectedMustPassAll =
1725      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1726        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1727        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1728        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
1729    verifyScanFull(scan, expectedMustPassAll);
1730    // 9. Test specified columns with FilterList(MUST_PASS_ONE) which sused in case 7.
1731    // Result is different from case 7, because column is strongly constrained by specified columns
1732    Scan anotherScan = new Scan().addColumn(FAMILIES[1], QUALIFIERS_FIVE[1])
1733      .setFilter(new FilterList(Operator.MUST_PASS_ONE, orFilters));
1734    KeyValue[] expectedValues =
1735      { new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1736        new KeyValue(ROWS_TWO[1], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1737        new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]),
1738        new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_FIVE[1], VALUES[0]) };
1739    verifyScanFull(anotherScan, expectedValues);
1740  }
1741
1742  private void verifyScan(Scan s, long expectedRows, long expectedKeys) throws IOException {
1743    InternalScanner scanner = this.region.getScanner(s);
1744    List<Cell> results = new ArrayList<>();
1745    int i = 0;
1746    for (boolean done = true; done; i++) {
1747      done = scanner.next(results);
1748      Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance());
1749      LOG.info("counter=" + i + ", " + results);
1750      if (results.isEmpty()) break;
1751      assertTrue(expectedRows > i, "Scanned too many rows! Only expected " + expectedRows
1752        + " total but already scanned " + (i + 1));
1753      assertEquals(expectedKeys, results.size(),
1754        "Expected " + expectedKeys + " keys per row but " + "returned " + results.size());
1755      results.clear();
1756    }
1757    assertEquals(expectedRows, i, "Expected " + expectedRows + " rows but scanned " + i + " rows");
1758  }
1759
1760  private void verifyScanNoEarlyOut(Scan s, long expectedRows, long expectedKeys)
1761    throws IOException {
1762    InternalScanner scanner = this.region.getScanner(s);
1763    List<Cell> results = new ArrayList<>();
1764    int i = 0;
1765    for (boolean done = true; done; i++) {
1766      done = scanner.next(results);
1767      Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance());
1768      LOG.info("counter=" + i + ", " + results);
1769      if (results.isEmpty()) break;
1770      assertTrue(expectedRows > i, "Scanned too many rows! Only expected " + expectedRows
1771        + " total but already scanned " + (i + 1));
1772      assertEquals(expectedKeys, results.size(),
1773        "Expected " + expectedKeys + " keys per row but " + "returned " + results.size());
1774      results.clear();
1775    }
1776    assertEquals(expectedRows, i, "Expected " + expectedRows + " rows but scanned " + i + " rows");
1777  }
1778
1779  private void verifyScanFull(Scan s, KeyValue[] kvs) throws IOException {
1780    InternalScanner scanner = this.region.getScanner(s);
1781    List<Cell> results = new ArrayList<>();
1782    int row = 0;
1783    int idx = 0;
1784    for (boolean done = true; done; row++) {
1785      done = scanner.next(results);
1786      Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance());
1787      if (results.isEmpty()) break;
1788      assertTrue(kvs.length >= idx + results.size(),
1789        "Scanned too many keys! Only expected " + kvs.length + " total but already scanned "
1790          + (results.size() + idx)
1791          + (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"));
1792      for (Cell kv : results) {
1793        LOG.info("row=" + row + ", result=" + kv.toString() + ", match=" + kvs[idx].toString());
1794        assertTrue(CellUtil.matchingRows(kv, kvs[idx]), "Row mismatch");
1795        assertTrue(CellUtil.matchingFamily(kv, kvs[idx]), "Family mismatch");
1796        assertTrue(CellUtil.matchingQualifier(kv, kvs[idx]), "Qualifier mismatch");
1797        assertTrue(CellUtil.matchingValue(kv, kvs[idx]), "Value mismatch");
1798        idx++;
1799      }
1800      results.clear();
1801    }
1802    LOG.info("Looked at " + row + " rows with " + idx + " keys");
1803    assertEquals(kvs.length, idx, "Expected " + kvs.length + " total keys but scanned " + idx);
1804  }
1805
1806  private void verifyScanFullNoValues(Scan s, KeyValue[] kvs, boolean useLen) throws IOException {
1807    InternalScanner scanner = this.region.getScanner(s);
1808    List<Cell> results = new ArrayList<>();
1809    int row = 0;
1810    int idx = 0;
1811    for (boolean more = true; more; row++) {
1812      more = scanner.next(results);
1813      Arrays.sort(results.toArray(new Cell[results.size()]), CellComparator.getInstance());
1814      if (results.isEmpty()) break;
1815      assertTrue(kvs.length >= idx + results.size(),
1816        "Scanned too many keys! Only expected " + kvs.length + " total but already scanned "
1817          + (results.size() + idx)
1818          + (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"));
1819      for (Cell kv : results) {
1820        LOG.info("row=" + row + ", result=" + kv.toString() + ", match=" + kvs[idx].toString());
1821
1822        assertTrue(CellUtil.matchingRows(kv, kvs[idx]), "Row mismatch");
1823        assertTrue(CellUtil.matchingFamily(kv, kvs[idx]), "Family mismatch");
1824        assertTrue(CellUtil.matchingQualifier(kv, kvs[idx]), "Qualifier mismatch");
1825        assertFalse(CellUtil.matchingValue(kv, kvs[idx]), "Should not have returned whole value");
1826        if (useLen) {
1827          assertEquals(Bytes.SIZEOF_INT, kv.getValueLength(), "Value in result is not SIZEOF_INT");
1828          LOG.info("idx = " + idx + ", len=" + kvs[idx].getValueLength() + ", actual="
1829            + Bytes.toInt(CellUtil.cloneValue(kv)));
1830          assertEquals(kvs[idx].getValueLength(), Bytes.toInt(CellUtil.cloneValue(kv)),
1831            "Scan value should be the length of the actual value. ");
1832          LOG.info("good");
1833        } else {
1834          assertEquals(0, kv.getValueLength(), "Value in result is not empty");
1835        }
1836        idx++;
1837      }
1838      results.clear();
1839    }
1840    LOG.info("Looked at " + row + " rows with " + idx + " keys");
1841    assertEquals(kvs.length, idx, "Expected " + kvs.length + " total keys but scanned " + idx);
1842  }
1843
1844  @Test
1845  public void testColumnPaginationFilterColumnOffset() throws Exception {
1846    KeyValue[] expectedKVs = {
1847      // testRowOne-0
1848      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1849      // testRowOne-2
1850      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1851      // testRowOne-3
1852      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1853      // testRowTwo-0
1854      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1855      // testRowTwo-2
1856      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1857      // testRowTwo-3
1858      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]), };
1859    KeyValue[] expectedKVs1 = {
1860      // testRowTwo-0
1861      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1862      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1863      // testRowTwo-2
1864      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1865      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1866      // testRowTwo-3
1867      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1868      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]) };
1869    KeyValue[] expectedKVs2 = {
1870      // testRowTwo-0
1871      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1872      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1873      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1874      // testRowTwo-2
1875      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1876      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1877      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1878      // testRowTwo-3
1879      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1880      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1881      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]) };
1882    KeyValue[] expectedKVs3 = {
1883      // testRowTwo-0
1884      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1885      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1886      // testRowTwo-2
1887      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1888      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1889      // testRowTwo-3
1890      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1891      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]), };
1892    Scan s = new Scan();
1893
1894    // Page size 1.
1895    long expectedRows = 6;
1896    long expectedKeys = 1;
1897    s.setFilter(new ColumnPaginationFilter(1, QUALIFIERS_ONE[1]));
1898    verifyScan(s, expectedRows, expectedKeys);
1899    this.verifyScanFull(s, expectedKVs);
1900
1901    // Page size 2.
1902    expectedRows = 3;
1903    expectedKeys = 2;
1904    s = new Scan();
1905    s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
1906    verifyScan(s, expectedRows, expectedKeys);
1907    this.verifyScanFull(s, expectedKVs1);
1908
1909    // Page size 3 across multiple column families.
1910    expectedRows = 3;
1911    expectedKeys = 3;
1912    s = new Scan();
1913    s.setFilter(new ColumnPaginationFilter(3, QUALIFIERS_TWO[2]));
1914    verifyScan(s, expectedRows, expectedKeys);
1915    this.verifyScanFull(s, expectedKVs2);
1916
1917    // Page size 2 restricted to one column family.
1918    expectedRows = 3;
1919    expectedKeys = 2;
1920    s = new Scan();
1921    s.addFamily(FAMILIES[1]);
1922    s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
1923    this.verifyScanFull(s, expectedKVs3);
1924  }
1925
1926  @Test
1927  public void testLatestVersionFilterWithExplicitColumn() throws Exception {
1928    // Add multiple versions
1929    Put p = new Put(ROWS_ONE[0]);
1930    p.setDurability(Durability.SKIP_WAL);
1931    p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]);
1932    this.region.put(p);
1933    p = new Put(ROWS_ONE[0]);
1934    p.setDurability(Durability.SKIP_WAL);
1935    p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]);
1936    this.region.put(p);
1937    this.region.flush(true);
1938    Scan s = new Scan();
1939    s.setFilter(new FilterBase() {
1940      @Override
1941      public ReturnCode filterCell(Cell c) throws IOException {
1942        return ReturnCode.INCLUDE_AND_NEXT_COL;
1943      }
1944    });
1945    s.readVersions(100);
1946    s.addColumn(FAMILIES[0], QUALIFIERS_ONE[0]);
1947    s.addColumn(FAMILIES[0], QUALIFIERS_ONE[1]);
1948    s.addColumn(FAMILIES[0], QUALIFIERS_ONE[2]);
1949    s.addColumn(FAMILIES[0], QUALIFIERS_ONE[3]);
1950    s.addColumn(FAMILIES[0], QUALIFIERS_TWO[0]);
1951    s.addColumn(FAMILIES[0], QUALIFIERS_TWO[1]);
1952    s.addColumn(FAMILIES[0], QUALIFIERS_TWO[2]);
1953    s.addColumn(FAMILIES[0], QUALIFIERS_TWO[3]);
1954    KeyValue[] kvs = {
1955      // testRowOne-0
1956      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
1957      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1958      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1959
1960      // testRowOne-2
1961      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1962      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1963      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1964
1965      // testRowOne-3
1966      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1967      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1968      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1969      // testRowTwo-0
1970      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1971      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1972      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1973      // testRowTwo-2
1974      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1975      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1976      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1977      // testRowTwo-3
1978      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1979      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1980      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), };
1981    verifyScanFull(s, kvs);
1982
1983  }
1984
1985  @Test
1986  public void testColumnPaginationFilter() throws Exception {
1987    // Test that the filter skips multiple column versions.
1988    Put p = new Put(ROWS_ONE[0]);
1989    p.setDurability(Durability.SKIP_WAL);
1990    p.addColumn(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]);
1991    this.region.put(p);
1992    this.region.flush(true);
1993
1994    // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
1995    KeyValue[] expectedKVs = {
1996      // testRowOne-0
1997      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1998      // testRowOne-2
1999      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
2000      // testRowOne-3
2001      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
2002      // testRowTwo-0
2003      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
2004      // testRowTwo-2
2005      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
2006      // testRowTwo-3
2007      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]) };
2008
2009    // Set of KVs (page: 3; pageSize: 1) - the third set of 1 column per row
2010    KeyValue[] expectedKVs2 = {
2011      // testRowOne-0
2012      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2013      // testRowOne-2
2014      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2015      // testRowOne-3
2016      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2017      // testRowTwo-0
2018      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2019      // testRowTwo-2
2020      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2021      // testRowTwo-3
2022      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]), };
2023
2024    // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row
2025    KeyValue[] expectedKVs3 = {
2026      // testRowOne-0
2027      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2028      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2029      // testRowOne-2
2030      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2031      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2032      // testRowOne-3
2033      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2034      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2035      // testRowTwo-0
2036      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2037      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
2038      // testRowTwo-2
2039      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2040      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
2041      // testRowTwo-3
2042      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2043      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]), };
2044
2045    // Set of KVs (page: 2; pageSize 2) - the 2nd set of 2 columns per row
2046    KeyValue[] expectedKVs4 = {
2047
2048    };
2049
2050    long expectedRows = this.numRows;
2051    long expectedKeys = 1;
2052    Scan s = new Scan();
2053
2054    // Page 1; 1 Column per page (Limit 1, Offset 0)
2055    s.setFilter(new ColumnPaginationFilter(1, 0));
2056    verifyScan(s, expectedRows, expectedKeys);
2057    this.verifyScanFull(s, expectedKVs);
2058
2059    // Page 3; 1 Result per page (Limit 1, Offset 2)
2060    s.setFilter(new ColumnPaginationFilter(1, 2));
2061    verifyScan(s, expectedRows, expectedKeys);
2062    this.verifyScanFull(s, expectedKVs2);
2063
2064    // Page 2; 2 Results per page (Limit 2, Offset 2)
2065    s.setFilter(new ColumnPaginationFilter(2, 2));
2066    expectedKeys = 2;
2067    verifyScan(s, expectedRows, expectedKeys);
2068    this.verifyScanFull(s, expectedKVs3);
2069
2070    // Page 8; 20 Results per page (no results) (Limit 20, Offset 140)
2071    s.setFilter(new ColumnPaginationFilter(20, 140));
2072    expectedKeys = 0;
2073    expectedRows = 0;
2074    verifyScan(s, expectedRows, 0);
2075    this.verifyScanFull(s, expectedKVs4);
2076  }
2077
2078  @Test
2079  public void testKeyOnlyFilter() throws Exception {
2080
2081    // KVs in first 6 rows
2082    KeyValue[] expectedKVs = {
2083      // testRowOne-0
2084      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
2085      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
2086      new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2087      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2088      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
2089      new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
2090      // testRowOne-2
2091      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
2092      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
2093      new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2094      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2095      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
2096      new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
2097      // testRowOne-3
2098      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
2099      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
2100      new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
2101      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
2102      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
2103      new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
2104      // testRowTwo-0
2105      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
2106      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
2107      new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2108      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
2109      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
2110      new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
2111      // testRowTwo-2
2112      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
2113      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
2114      new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2115      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
2116      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
2117      new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
2118      // testRowTwo-3
2119      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
2120      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
2121      new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
2122      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
2123      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
2124      new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]) };
2125
2126    // Grab all 6 rows
2127    long expectedRows = 6;
2128    long expectedKeys = this.colsPerRow;
2129    for (boolean useLen : new boolean[] { false, true }) {
2130      Scan s = new Scan();
2131      s.setFilter(new KeyOnlyFilter(useLen));
2132      verifyScan(s, expectedRows, expectedKeys);
2133      verifyScanFullNoValues(s, expectedKVs, useLen);
2134    }
2135  }
2136
2137  /**
2138   * Filter which makes sleeps for a second between each row of a scan. This can be useful for
2139   * manual testing of bugs like HBASE-5973. For example: <code>
2140   * create 't1', 'f1'
2141   * 1.upto(100)  { |x| put 't1', 'r' + x.to_s, 'f1:q1', 'hi' }
2142   * import org.apache.hadoop.hbase.filter.TestFilter
2143   * scan 't1', { FILTER => TestFilter::SlowScanFilter.new(), CACHE => 50 }
2144   * </code>
2145   */
2146  public static class SlowScanFilter extends FilterBase {
2147    private static Thread ipcHandlerThread = null;
2148
2149    @Override
2150    public byte[] toByteArray() {
2151      return null;
2152    }
2153
2154    @Override
2155    public ReturnCode filterCell(final Cell ignored) throws IOException {
2156      return ReturnCode.INCLUDE;
2157    }
2158
2159    @Override
2160    public boolean filterRow() throws IOException {
2161      ipcHandlerThread = Thread.currentThread();
2162      try {
2163        LOG.info("Handler thread " + ipcHandlerThread + " sleeping in filter...");
2164        Thread.sleep(1000);
2165      } catch (InterruptedException e) {
2166        Throwables.propagate(e);
2167      }
2168      return super.filterRow();
2169    }
2170  }
2171
2172  @Test
2173  @Disabled("TODO: intentionally disabled?")
2174  public void testNestedFilterListWithSCVF(TestInfo testInfo) throws IOException {
2175    byte[] columnStatus = Bytes.toBytes("S");
2176    TableDescriptor tableDescriptor =
2177      TableDescriptorBuilder.newBuilder(TableName.valueOf(testInfo.getTestMethod().get().getName()))
2178        .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILIES[0])).build();
2179    RegionInfo info = RegionInfoBuilder.newBuilder(tableDescriptor.getTableName()).build();
2180    HRegion testRegion = HBaseTestingUtil.createRegionAndWAL(info, TEST_UTIL.getDataTestDir(),
2181      TEST_UTIL.getConfiguration(), tableDescriptor);
2182    for (int i = 0; i < 10; i++) {
2183      Put p = new Put(Bytes.toBytes("row" + i));
2184      p.setDurability(Durability.SKIP_WAL);
2185      p.addColumn(FAMILIES[0], columnStatus, Bytes.toBytes(i % 2));
2186      testRegion.put(p);
2187    }
2188    testRegion.flush(true);
2189    // 1. got rows > "row4"
2190    Filter rowFilter =
2191      new RowFilter(CompareOperator.GREATER, new BinaryComparator(Bytes.toBytes("row4")));
2192    Scan s1 = new Scan();
2193    s1.setFilter(rowFilter);
2194    InternalScanner scanner = testRegion.getScanner(s1);
2195    List<Cell> results = new ArrayList<>();
2196    int i = 5;
2197    for (boolean done = true; done; i++) {
2198      done = scanner.next(results);
2199      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2200      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2201      results.clear();
2202    }
2203    // 2. got rows <= "row4" and S=
2204    FilterList subFilterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
2205    Filter subFilter1 =
2206      new RowFilter(CompareOperator.LESS_OR_EQUAL, new BinaryComparator(Bytes.toBytes("row4")));
2207    subFilterList.addFilter(subFilter1);
2208    Filter subFilter2 = new SingleColumnValueFilter(FAMILIES[0], columnStatus,
2209      CompareOperator.EQUAL, Bytes.toBytes(0));
2210    subFilterList.addFilter(subFilter2);
2211    s1 = new Scan();
2212    s1.setFilter(subFilterList);
2213    scanner = testRegion.getScanner(s1);
2214    results = new ArrayList<>();
2215    for (i = 0; i <= 4; i += 2) {
2216      scanner.next(results);
2217      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2218      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2219      results.clear();
2220    }
2221    assertFalse(scanner.next(results));
2222    // 3. let's begin to verify nested filter list
2223    // 3.1 add rowFilter, then add subFilterList
2224    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
2225    filterList.addFilter(rowFilter);
2226    filterList.addFilter(subFilterList);
2227    s1 = new Scan();
2228    s1.setFilter(filterList);
2229    scanner = testRegion.getScanner(s1);
2230    results = new ArrayList<>();
2231    for (i = 0; i <= 4; i += 2) {
2232      scanner.next(results);
2233      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2234      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2235      results.clear();
2236    }
2237    for (i = 5; i <= 9; i++) {
2238      scanner.next(results);
2239      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2240      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2241      results.clear();
2242    }
2243    assertFalse(scanner.next(results));
2244    // 3.2 MAGIC here! add subFilterList first, then add rowFilter
2245    filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
2246    filterList.addFilter(subFilterList);
2247    filterList.addFilter(rowFilter);
2248    s1 = new Scan();
2249    s1.setFilter(filterList);
2250    scanner = testRegion.getScanner(s1);
2251    results = new ArrayList<>();
2252    for (i = 0; i <= 4; i += 2) {
2253      scanner.next(results);
2254      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2255      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2256      results.clear();
2257    }
2258    for (i = 5; i <= 9; i++) {
2259      scanner.next(results);
2260      assertTrue(CellUtil.matchingRows(results.get(0), Bytes.toBytes("row" + i)));
2261      assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i % 2);
2262      results.clear();
2263    }
2264    assertFalse(scanner.next(results));
2265    WAL wal = ((HRegion) testRegion).getWAL();
2266    ((HRegion) testRegion).close();
2267    wal.close();
2268  }
2269}