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