View Javadoc

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