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