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