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.TableName;
42  import org.apache.hadoop.hbase.client.Delete;
43  import org.apache.hadoop.hbase.client.Durability;
44  import org.apache.hadoop.hbase.client.Put;
45  import org.apache.hadoop.hbase.client.Scan;
46  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
47  import org.apache.hadoop.hbase.filter.FilterList.Operator;
48  import org.apache.hadoop.hbase.regionserver.HRegion;
49  import org.apache.hadoop.hbase.regionserver.InternalScanner;
50  import org.apache.hadoop.hbase.regionserver.Region;
51  import org.apache.hadoop.hbase.regionserver.RegionScanner;
52  import org.apache.hadoop.hbase.testclassification.SmallTests;
53  import org.apache.hadoop.hbase.util.Bytes;
54  import org.apache.hadoop.hbase.wal.WAL;
55  import org.junit.After;
56  import org.junit.Assert;
57  import org.junit.Before;
58  import org.junit.Test;
59  import org.junit.experimental.categories.Category;
60  
61  import com.google.common.base.Throwables;
62  
63  /**
64   * Test filters at the HRegion doorstep.
65   */
66  @Category(SmallTests.class)
67  public class TestFilter {
68    private final static Log LOG = LogFactory.getLog(TestFilter.class);
69    private Region region;
70    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
71  
72    //
73    // Rows, Qualifiers, and Values are in two groups, One and Two.
74    //
75  
76    private static final byte [][] ROWS_ONE = {
77        Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
78        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3")
79    };
80  
81    private static final byte [][] ROWS_TWO = {
82        Bytes.toBytes("testRowTwo-0"), Bytes.toBytes("testRowTwo-1"),
83        Bytes.toBytes("testRowTwo-2"), Bytes.toBytes("testRowTwo-3")
84    };
85  
86    private static final byte [][] ROWS_THREE = {
87      Bytes.toBytes("testRowThree-0"), Bytes.toBytes("testRowThree-1"),
88      Bytes.toBytes("testRowThree-2"), Bytes.toBytes("testRowThree-3")
89    };
90  
91    private static final byte [][] ROWS_FOUR = {
92      Bytes.toBytes("testRowFour-0"), Bytes.toBytes("testRowFour-1"),
93      Bytes.toBytes("testRowFour-2"), Bytes.toBytes("testRowFour-3")
94    };
95  
96    private static final byte [][] FAMILIES = {
97      Bytes.toBytes("testFamilyOne"), Bytes.toBytes("testFamilyTwo")
98    };
99  
100   private static final byte [][] FAMILIES_1 = {
101     Bytes.toBytes("testFamilyThree"), Bytes.toBytes("testFamilyFour")
102   };
103 
104   private static final byte [][] QUALIFIERS_ONE = {
105     Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"),
106     Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3")
107   };
108 
109   private static final byte [][] QUALIFIERS_TWO = {
110     Bytes.toBytes("testQualifierTwo-0"), Bytes.toBytes("testQualifierTwo-1"),
111     Bytes.toBytes("testQualifierTwo-2"), Bytes.toBytes("testQualifierTwo-3")
112   };
113 
114   private static final byte [][] QUALIFIERS_THREE = {
115     Bytes.toBytes("testQualifierThree-0"), Bytes.toBytes("testQualifierThree-1"),
116     Bytes.toBytes("testQualifierThree-2"), Bytes.toBytes("testQualifierThree-3")
117   };
118 
119   private static final byte [][] QUALIFIERS_FOUR = {
120     Bytes.toBytes("testQualifierFour-0"), Bytes.toBytes("testQualifierFour-1"),
121     Bytes.toBytes("testQualifierFour-2"), Bytes.toBytes("testQualifierFour-3")
122   };
123 
124   private static final byte [][] VALUES = {
125     Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo")
126   };
127 
128   byte [][] NEW_FAMILIES = {
129       Bytes.toBytes("f1"), Bytes.toBytes("f2")
130     };
131 
132   private long numRows = ROWS_ONE.length + ROWS_TWO.length;
133   private long colsPerRow = FAMILIES.length * QUALIFIERS_ONE.length;
134 
135   @Before
136   public void setUp() throws Exception {
137     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TestFilter"));
138     htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
139     htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
140     htd.addFamily(new HColumnDescriptor(FAMILIES_1[0]));
141     htd.addFamily(new HColumnDescriptor(FAMILIES_1[1]));
142     htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[0]));
143     htd.addFamily(new HColumnDescriptor(NEW_FAMILIES[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.flush(true);
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     WAL wal = ((HRegion)region).getWAL();
221     ((HRegion)region).close();
222     wal.close();
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.flush(true);
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     @Override
603     public ReturnCode filterKeyValue(Cell ignored) throws IOException {
604       return ReturnCode.INCLUDE;
605     }
606   }
607   
608   /**
609    * The following test is to ensure old(such as hbase0.94) filterRow() can be correctly fired in 
610    * 0.96+ code base.  
611    * 
612    * See HBASE-10366
613    * 
614    * @throws Exception
615    */
616   @Test
617   public void tes94FilterRowCompatibility() throws Exception {
618     Scan s = new Scan();
619     OldTestFilter filter = new OldTestFilter();
620     s.setFilter(filter);
621 
622     InternalScanner scanner = this.region.getScanner(s);
623     ArrayList<Cell> values = new ArrayList<Cell>();
624     scanner.next(values);
625     assertTrue("All rows should be filtered out", values.isEmpty());
626   }
627 
628   /**
629    * Tests the the {@link WhileMatchFilter} works in combination with a
630    * {@link Filter} that uses the
631    * {@link Filter#filterRowKey(byte[], int, int)} method.
632    *
633    * See HBASE-2258.
634    *
635    * @throws Exception
636    */
637   @Test
638   public void testWhileMatchFilterWithFilterRowKey() throws Exception {
639     Scan s = new Scan();
640     String prefix = "testRowOne";
641     WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix)));
642     s.setFilter(filter);
643 
644     InternalScanner scanner = this.region.getScanner(s);
645     while (true) {
646       ArrayList<Cell> values = new ArrayList<Cell>();
647       boolean isMoreResults = scanner.next(values);
648       if (!isMoreResults || !Bytes.toString(CellUtil.cloneRow(values.get(0))).startsWith(prefix)) {
649         assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
650       }
651       if (!isMoreResults) {
652         break;
653       }
654     }
655   }
656 
657   /**
658    * Tests the the {@link WhileMatchFilter} works in combination with a
659    * {@link Filter} that uses the
660    * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)} method.
661    *
662    * See HBASE-2258.
663    *
664    * @throws Exception
665    */
666   @Test
667   public void testWhileMatchFilterWithFilterKeyValue() throws Exception {
668     Scan s = new Scan();
669     WhileMatchFilter filter = new WhileMatchFilter(
670         new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOp.EQUAL, Bytes.toBytes("foo"))
671     );
672     s.setFilter(filter);
673 
674     InternalScanner scanner = this.region.getScanner(s);
675     while (true) {
676       ArrayList<Cell> values = new ArrayList<Cell>();
677       boolean isMoreResults = scanner.next(values);
678       assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
679       if (!isMoreResults) {
680         break;
681       }
682     }
683   }
684 
685   @Test
686   public void testInclusiveStopFilter() throws IOException {
687 
688     // Grab rows from group one
689 
690     // If we just use start/stop row, we get total/2 - 1 rows
691     long expectedRows = (this.numRows / 2) - 1;
692     long expectedKeys = this.colsPerRow;
693     Scan s = new Scan(Bytes.toBytes("testRowOne-0"),
694         Bytes.toBytes("testRowOne-3"));
695     verifyScan(s, expectedRows, expectedKeys);
696 
697     // Now use start row with inclusive stop filter
698     expectedRows = this.numRows / 2;
699     s = new Scan(Bytes.toBytes("testRowOne-0"));
700     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-3")));
701     verifyScan(s, expectedRows, expectedKeys);
702 
703     // Grab rows from group two
704 
705     // If we just use start/stop row, we get total/2 - 1 rows
706     expectedRows = (this.numRows / 2) - 1;
707     expectedKeys = this.colsPerRow;
708     s = new Scan(Bytes.toBytes("testRowTwo-0"),
709         Bytes.toBytes("testRowTwo-3"));
710     verifyScan(s, expectedRows, expectedKeys);
711 
712     // Now use start row with inclusive stop filter
713     expectedRows = this.numRows / 2;
714     s = new Scan(Bytes.toBytes("testRowTwo-0"));
715     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-3")));
716     verifyScan(s, expectedRows, expectedKeys);
717 
718   }
719 
720   @Test
721   public void testQualifierFilter() throws IOException {
722 
723     // Match two keys (one from each family) in half the rows
724     long expectedRows = this.numRows / 2;
725     long expectedKeys = 2;
726     Filter f = new QualifierFilter(CompareOp.EQUAL,
727         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
728     Scan s = new Scan();
729     s.setFilter(f);
730     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
731 
732     // Match keys less than same qualifier
733     // Expect only two keys (one from each family) in half the rows
734     expectedRows = this.numRows / 2;
735     expectedKeys = 2;
736     f = new QualifierFilter(CompareOp.LESS,
737         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
738     s = new Scan();
739     s.setFilter(f);
740     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
741 
742     // Match keys less than or equal
743     // Expect four keys (two from each family) in half the rows
744     expectedRows = this.numRows / 2;
745     expectedKeys = 4;
746     f = new QualifierFilter(CompareOp.LESS_OR_EQUAL,
747         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
748     s = new Scan();
749     s.setFilter(f);
750     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
751 
752     // Match keys not equal
753     // Expect four keys (two from each family)
754     // Only look in first group of rows
755     expectedRows = this.numRows / 2;
756     expectedKeys = 4;
757     f = new QualifierFilter(CompareOp.NOT_EQUAL,
758         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
759     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
760     s.setFilter(f);
761     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
762 
763     // Match keys greater or equal
764     // Expect four keys (two from each family)
765     // Only look in first group of rows
766     expectedRows = this.numRows / 2;
767     expectedKeys = 4;
768     f = new QualifierFilter(CompareOp.GREATER_OR_EQUAL,
769         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
770     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
771     s.setFilter(f);
772     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
773 
774     // Match keys greater
775     // Expect two keys (one from each family)
776     // Only look in first group of rows
777     expectedRows = this.numRows / 2;
778     expectedKeys = 2;
779     f = new QualifierFilter(CompareOp.GREATER,
780         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
781     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
782     s.setFilter(f);
783     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
784 
785     // Match keys not equal to
786     // Look across rows and fully validate the keys and ordering
787     // Expect varied numbers of keys, 4 per row in group one, 6 per row in group two
788     f = new QualifierFilter(CompareOp.NOT_EQUAL,
789         new BinaryComparator(QUALIFIERS_ONE[2]));
790     s = new Scan();
791     s.setFilter(f);
792 
793     KeyValue [] kvs = {
794         // testRowOne-0
795         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
796         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
797         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
798         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
799         // testRowOne-2
800         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
801         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
802         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
803         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
804         // testRowOne-3
805         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
806         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
807         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
808         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
809         // testRowTwo-0
810         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
811         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
812         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
813         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
814         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
815         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
816         // testRowTwo-2
817         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
818         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
819         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
820         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
821         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
822         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
823         // testRowTwo-3
824         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
825         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
826         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
827         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
828         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
829         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
830     };
831     verifyScanFull(s, kvs);
832 
833 
834     // Test across rows and groups with a regex
835     // Filter out "test*-2"
836     // Expect 4 keys per row across both groups
837     f = new QualifierFilter(CompareOp.NOT_EQUAL,
838         new RegexStringComparator("test.+-2"));
839     s = new Scan();
840     s.setFilter(f);
841 
842     kvs = new KeyValue [] {
843         // testRowOne-0
844         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
845         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
846         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
847         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
848         // testRowOne-2
849         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
850         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
851         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
852         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
853         // testRowOne-3
854         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
855         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
856         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
857         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
858         // testRowTwo-0
859         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
860         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
861         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
862         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
863         // testRowTwo-2
864         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
865         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
866         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
867         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
868         // testRowTwo-3
869         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
870         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
871         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
872         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
873     };
874     verifyScanFull(s, kvs);
875 
876   }
877 
878   @Test
879   public void testFamilyFilter() throws IOException {
880 
881       // Match family, only half of columns returned.
882       long expectedRows = this.numRows;
883       long expectedKeys = this.colsPerRow / 2;
884       Filter f = new FamilyFilter(CompareOp.EQUAL,
885           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
886       Scan s = new Scan();
887       s.setFilter(f);
888       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
889 
890       // Match keys less than given family, should return nothing
891       expectedRows = 0;
892       expectedKeys = 0;
893       f = new FamilyFilter(CompareOp.LESS,
894           new BinaryComparator(Bytes.toBytes("testFamily")));
895       s = new Scan();
896       s.setFilter(f);
897       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
898 
899       // Match keys less than or equal, should return half of columns
900       expectedRows = this.numRows;
901       expectedKeys = this.colsPerRow / 2;
902       f = new FamilyFilter(CompareOp.LESS_OR_EQUAL,
903           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
904       s = new Scan();
905       s.setFilter(f);
906       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
907 
908       // Match keys from second family
909       // look only in second group of rows
910       expectedRows = this.numRows / 2;
911       expectedKeys = this.colsPerRow / 2;
912       f = new FamilyFilter(CompareOp.NOT_EQUAL,
913           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
914       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
915       s.setFilter(f);
916       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
917 
918       // Match all columns
919       // look only in second group of rows
920       expectedRows = this.numRows / 2;
921       expectedKeys = this.colsPerRow;
922       f = new FamilyFilter(CompareOp.GREATER_OR_EQUAL,
923           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
924       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
925       s.setFilter(f);
926       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
927 
928       // Match all columns in second family
929       // look only in second group of rows
930       expectedRows = this.numRows / 2;
931       expectedKeys = this.colsPerRow / 2;
932       f = new FamilyFilter(CompareOp.GREATER,
933           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
934       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
935       s.setFilter(f);
936       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
937 
938       // Match keys not equal to given family
939       // Look across rows and fully validate the keys and ordering
940       f = new FamilyFilter(CompareOp.NOT_EQUAL,
941           new BinaryComparator(FAMILIES[1]));
942       s = new Scan();
943       s.setFilter(f);
944 
945       KeyValue [] kvs = {
946           // testRowOne-0
947           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
948           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
949           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
950           // testRowOne-2
951           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
952           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
953           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
954           // testRowOne-3
955           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
956           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
957           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
958           // testRowTwo-0
959           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
960           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
961           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
962           // testRowTwo-2
963           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
964           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
965           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
966           // testRowTwo-3
967           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
968           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
969           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
970       };
971       verifyScanFull(s, kvs);
972 
973 
974       // Test across rows and groups with a regex
975       // Filter out "test*-2"
976       // Expect 4 keys per row across both groups
977       f = new FamilyFilter(CompareOp.NOT_EQUAL,
978           new RegexStringComparator("test.*One"));
979       s = new Scan();
980       s.setFilter(f);
981 
982       kvs = new KeyValue [] {
983           // testRowOne-0
984           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
985           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
986           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
987           // testRowOne-2
988           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
989           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
990           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
991           // testRowOne-3
992           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
993           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
994           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
995           // testRowTwo-0
996           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
997           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
998           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
999           // testRowTwo-2
1000           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1001           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1002           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1003           // testRowTwo-3
1004           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1005           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1006           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1007       };
1008       verifyScanFull(s, kvs);
1009 
1010     }
1011 
1012 
1013   @Test
1014   public void testRowFilter() throws IOException {
1015 
1016     // Match a single row, all keys
1017     long expectedRows = 1;
1018     long expectedKeys = this.colsPerRow;
1019     Filter f = new RowFilter(CompareOp.EQUAL,
1020         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1021     Scan s = new Scan();
1022     s.setFilter(f);
1023     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1024 
1025     // Match a two rows, one from each group, using regex
1026     expectedRows = 2;
1027     expectedKeys = this.colsPerRow;
1028     f = new RowFilter(CompareOp.EQUAL,
1029         new RegexStringComparator("testRow.+-2"));
1030     s = new Scan();
1031     s.setFilter(f);
1032     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1033 
1034     // Match rows less than
1035     // Expect all keys in one row
1036     expectedRows = 1;
1037     expectedKeys = this.colsPerRow;
1038     f = new RowFilter(CompareOp.LESS,
1039         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1040     s = new Scan();
1041     s.setFilter(f);
1042     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1043 
1044     // Match rows less than or equal
1045     // Expect all keys in two rows
1046     expectedRows = 2;
1047     expectedKeys = this.colsPerRow;
1048     f = new RowFilter(CompareOp.LESS_OR_EQUAL,
1049         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1050     s = new Scan();
1051     s.setFilter(f);
1052     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1053 
1054     // Match rows not equal
1055     // Expect all keys in all but one row
1056     expectedRows = this.numRows - 1;
1057     expectedKeys = this.colsPerRow;
1058     f = new RowFilter(CompareOp.NOT_EQUAL,
1059         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1060     s = new Scan();
1061     s.setFilter(f);
1062     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1063 
1064     // Match keys greater or equal
1065     // Expect all keys in all but one row
1066     expectedRows = this.numRows - 1;
1067     expectedKeys = this.colsPerRow;
1068     f = new RowFilter(CompareOp.GREATER_OR_EQUAL,
1069         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1070     s = new Scan();
1071     s.setFilter(f);
1072     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1073 
1074     // Match keys greater
1075     // Expect all keys in all but two rows
1076     expectedRows = this.numRows - 2;
1077     expectedKeys = this.colsPerRow;
1078     f = new RowFilter(CompareOp.GREATER,
1079         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1080     s = new Scan();
1081     s.setFilter(f);
1082     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1083 
1084     // Match rows not equal to testRowTwo-2
1085     // Look across rows and fully validate the keys and ordering
1086     // Should see all keys in all rows but testRowTwo-2
1087     f = new RowFilter(CompareOp.NOT_EQUAL,
1088         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
1089     s = new Scan();
1090     s.setFilter(f);
1091 
1092     KeyValue [] kvs = {
1093         // testRowOne-0
1094         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1095         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1096         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1097         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1098         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1099         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1100         // testRowOne-3
1101         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1102         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1103         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1104         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1105         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1106         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1107         // testRowTwo-0
1108         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1109         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1110         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1111         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1112         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1113         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1114         // testRowTwo-2
1115         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1116         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1117         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1118         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1119         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1120         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1121         // testRowTwo-3
1122         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1123         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1124         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1125         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1126         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1127         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1128     };
1129     verifyScanFull(s, kvs);
1130 
1131 
1132     // Test across rows and groups with a regex
1133     // Filter out everything that doesn't match "*-2"
1134     // Expect all keys in two rows
1135     f = new RowFilter(CompareOp.EQUAL,
1136         new RegexStringComparator(".+-2"));
1137     s = new Scan();
1138     s.setFilter(f);
1139 
1140     kvs = new KeyValue [] {
1141         // testRowOne-2
1142         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1143         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1144         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1145         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1146         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1147         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1148         // testRowTwo-2
1149         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1150         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1151         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1152         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1153         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1154         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
1155     };
1156     verifyScanFull(s, kvs);
1157 
1158   }
1159 
1160   @Test
1161   public void testValueFilter() throws IOException {
1162 
1163     // Match group one rows
1164     long expectedRows = this.numRows / 2;
1165     long expectedKeys = this.colsPerRow;
1166     Filter f = new ValueFilter(CompareOp.EQUAL,
1167         new BinaryComparator(Bytes.toBytes("testValueOne")));
1168     Scan s = new Scan();
1169     s.setFilter(f);
1170     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1171 
1172     // Match group two rows
1173     expectedRows = this.numRows / 2;
1174     expectedKeys = this.colsPerRow;
1175     f = new ValueFilter(CompareOp.EQUAL,
1176         new BinaryComparator(Bytes.toBytes("testValueTwo")));
1177     s = new Scan();
1178     s.setFilter(f);
1179     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1180 
1181     // Match all values using regex
1182     expectedRows = this.numRows;
1183     expectedKeys = this.colsPerRow;
1184     f = new ValueFilter(CompareOp.EQUAL,
1185         new RegexStringComparator("testValue((One)|(Two))"));
1186     s = new Scan();
1187     s.setFilter(f);
1188     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1189 
1190     // Match values less than
1191     // Expect group one rows
1192     expectedRows = this.numRows / 2;
1193     expectedKeys = this.colsPerRow;
1194     f = new ValueFilter(CompareOp.LESS,
1195         new BinaryComparator(Bytes.toBytes("testValueTwo")));
1196     s = new Scan();
1197     s.setFilter(f);
1198     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1199 
1200     // Match values less than or equal
1201     // Expect all rows
1202     expectedRows = this.numRows;
1203     expectedKeys = this.colsPerRow;
1204     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
1205         new BinaryComparator(Bytes.toBytes("testValueTwo")));
1206     s = new Scan();
1207     s.setFilter(f);
1208     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1209 
1210     // Match values less than or equal
1211     // Expect group one rows
1212     expectedRows = this.numRows / 2;
1213     expectedKeys = this.colsPerRow;
1214     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
1215         new BinaryComparator(Bytes.toBytes("testValueOne")));
1216     s = new Scan();
1217     s.setFilter(f);
1218     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1219 
1220     // Match values not equal
1221     // Expect half the rows
1222     expectedRows = this.numRows / 2;
1223     expectedKeys = this.colsPerRow;
1224     f = new ValueFilter(CompareOp.NOT_EQUAL,
1225         new BinaryComparator(Bytes.toBytes("testValueOne")));
1226     s = new Scan();
1227     s.setFilter(f);
1228     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1229 
1230     // Match values greater or equal
1231     // Expect all rows
1232     expectedRows = this.numRows;
1233     expectedKeys = this.colsPerRow;
1234     f = new ValueFilter(CompareOp.GREATER_OR_EQUAL,
1235         new BinaryComparator(Bytes.toBytes("testValueOne")));
1236     s = new Scan();
1237     s.setFilter(f);
1238     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1239 
1240     // Match values greater
1241     // Expect half rows
1242     expectedRows = this.numRows / 2;
1243     expectedKeys = this.colsPerRow;
1244     f = new ValueFilter(CompareOp.GREATER,
1245         new BinaryComparator(Bytes.toBytes("testValueOne")));
1246     s = new Scan();
1247     s.setFilter(f);
1248     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
1249 
1250     // Match values not equal to testValueOne
1251     // Look across rows and fully validate the keys and ordering
1252     // Should see all keys in all group two rows
1253     f = new ValueFilter(CompareOp.NOT_EQUAL,
1254         new BinaryComparator(Bytes.toBytes("testValueOne")));
1255     s = new Scan();
1256     s.setFilter(f);
1257 
1258     KeyValue [] kvs = {
1259         // testRowTwo-0
1260         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1261         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1262         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1263         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1264         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1265         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1266         // testRowTwo-2
1267         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1268         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1269         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1270         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1271         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1272         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1273         // testRowTwo-3
1274         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1275         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1276         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1277         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1278         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1279         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1280     };
1281     verifyScanFull(s, kvs);
1282   }
1283 
1284   @Test
1285   public void testSkipFilter() throws IOException {
1286 
1287     // Test for qualifier regex: "testQualifierOne-2"
1288     // Should only get rows from second group, and all keys
1289     Filter f = new SkipFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
1290         new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))));
1291     Scan s = new Scan();
1292     s.setFilter(f);
1293 
1294     KeyValue [] kvs = {
1295         // testRowTwo-0
1296         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1297         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1298         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1299         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1300         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1301         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1302         // testRowTwo-2
1303         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1304         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1305         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1306         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1307         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1308         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1309         // testRowTwo-3
1310         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1311         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1312         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1313         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1314         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1315         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1316     };
1317     verifyScanFull(s, kvs);
1318   }
1319 
1320   // TODO: This is important... need many more tests for ordering, etc
1321   // There are limited tests elsewhere but we need HRegion level ones here
1322   @Test
1323   public void testFilterList() throws IOException {
1324 
1325     // Test getting a single row, single key using Row, Qualifier, and Value
1326     // regular expression and substring filters
1327     // Use must pass all
1328     List<Filter> filters = new ArrayList<Filter>();
1329     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1330     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1331     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
1332     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1333     Scan s = new Scan();
1334     s.addFamily(FAMILIES[0]);
1335     s.setFilter(f);
1336     KeyValue [] kvs = {
1337         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0])
1338     };
1339     verifyScanFull(s, kvs);
1340 
1341     // Test getting everything with a MUST_PASS_ONE filter including row, qf, val
1342     // regular expression and substring filters
1343     filters.clear();
1344     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+Two.+")));
1345     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1346     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
1347     f = new FilterList(Operator.MUST_PASS_ONE, filters);
1348     s = new Scan();
1349     s.setFilter(f);
1350     verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow);
1351 
1352 
1353   }
1354 
1355   @Test
1356   public void testFirstKeyOnlyFilter() throws IOException {
1357     Scan s = new Scan();
1358     s.setFilter(new FirstKeyOnlyFilter());
1359     // Expected KVs, the first KV from each of the remaining 6 rows
1360     KeyValue [] kvs = {
1361         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1362         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1363         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1364         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1365         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1366         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
1367     };
1368     verifyScanFull(s, kvs);
1369   }
1370 
1371   @Test
1372   public void testFilterListWithSingleColumnValueFilter() throws IOException {
1373     // Test for HBASE-3191
1374 
1375     // Scan using SingleColumnValueFilter
1376     SingleColumnValueFilter f1 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1377           CompareOp.EQUAL, VALUES[0]);
1378     f1.setFilterIfMissing( true );
1379     Scan s1 = new Scan();
1380     s1.addFamily(FAMILIES[0]);
1381     s1.setFilter(f1);
1382     KeyValue [] kvs1 = {
1383         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1384         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1385         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1386         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1387         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1388         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1389         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1390         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1391         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1392     };
1393     verifyScanNoEarlyOut(s1, 3, 3);
1394     verifyScanFull(s1, kvs1);
1395 
1396     // Scan using another SingleColumnValueFilter, expect disjoint result
1397     SingleColumnValueFilter f2 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_TWO[0],
1398         CompareOp.EQUAL, VALUES[1]);
1399     f2.setFilterIfMissing( true );
1400     Scan s2 = new Scan();
1401     s2.addFamily(FAMILIES[0]);
1402     s2.setFilter(f2);
1403     KeyValue [] kvs2 = {
1404         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1405         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1406         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1407         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1408         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1409         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1410         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1411         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1412         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1413     };
1414     verifyScanNoEarlyOut(s2, 3, 3);
1415     verifyScanFull(s2, kvs2);
1416 
1417     // Scan, ORing the two previous filters, expect unified result
1418     FilterList f = new FilterList(Operator.MUST_PASS_ONE);
1419     f.addFilter(f1);
1420     f.addFilter(f2);
1421     Scan s = new Scan();
1422     s.addFamily(FAMILIES[0]);
1423     s.setFilter(f);
1424     KeyValue [] kvs = {
1425         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1426         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1427         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1428         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1429         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1430         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1431         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1432         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1433         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1434         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1435         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1436         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1437         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1438         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1439         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1440         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1441         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1442         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1443     };
1444     verifyScanNoEarlyOut(s, 6, 3);
1445     verifyScanFull(s, kvs);
1446   }
1447 
1448   // HBASE-9747
1449   @Test
1450   public void testFilterListWithPrefixFilter() throws IOException {
1451     byte[] family = Bytes.toBytes("f1");
1452     byte[] qualifier = Bytes.toBytes("q1");
1453     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("TestFilter"));
1454     htd.addFamily(new HColumnDescriptor(family));
1455     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
1456     HRegion testRegion = HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(),
1457         TEST_UTIL.getConfiguration(), htd);
1458 
1459     for(int i=0; i<5; i++) {
1460       Put p = new Put(Bytes.toBytes((char)('a'+i) + "row"));
1461       p.setDurability(Durability.SKIP_WAL);
1462       p.add(family, qualifier, Bytes.toBytes(String.valueOf(111+i)));
1463       testRegion.put(p);
1464     }
1465     testRegion.flush(true);
1466 
1467     // rows starting with "b"
1468     PrefixFilter pf = new PrefixFilter(new byte[] {'b'}) ;
1469     // rows with value of column 'q1' set to '113'
1470     SingleColumnValueFilter scvf = new SingleColumnValueFilter(
1471         family, qualifier, CompareOp.EQUAL, Bytes.toBytes("113"));
1472     // combine these two with OR in a FilterList
1473     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, pf, scvf);
1474 
1475     Scan s1 = new Scan();
1476     s1.setFilter(filterList);
1477     InternalScanner scanner = testRegion.getScanner(s1);
1478     List<Cell> results = new ArrayList<Cell>();
1479     int resultCount = 0;
1480     while (scanner.next(results)) {
1481       resultCount++;
1482       byte[] row =  CellUtil.cloneRow(results.get(0));
1483       LOG.debug("Found row: " + Bytes.toStringBinary(row));
1484       assertTrue(Bytes.equals(row, Bytes.toBytes("brow"))
1485           || Bytes.equals(row, Bytes.toBytes("crow")));
1486       results.clear();
1487     }
1488     assertEquals(2, resultCount);
1489     scanner.close();
1490 
1491     WAL wal = ((HRegion)testRegion).getWAL();
1492     ((HRegion)testRegion).close();
1493     wal.close();
1494   }
1495 
1496   @Test
1497   public void testSingleColumnValueFilter() throws IOException {
1498 
1499     // From HBASE-1821
1500     // Desired action is to combine two SCVF in a FilterList
1501     // Want to return only rows that match both conditions
1502 
1503     // Need to change one of the group one columns to use group two value
1504     Put p = new Put(ROWS_ONE[2]);
1505     p.add(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]);
1506     this.region.put(p);
1507 
1508     // Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1])
1509     // Since group two rows don't have these qualifiers, they will pass
1510     // so limiting scan to group one
1511     List<Filter> filters = new ArrayList<Filter>();
1512     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1513         CompareOp.EQUAL, VALUES[0]));
1514     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
1515         CompareOp.EQUAL, VALUES[1]));
1516     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1517     Scan s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
1518     s.addFamily(FAMILIES[0]);
1519     s.setFilter(f);
1520     // Expect only one row, all qualifiers
1521     KeyValue [] kvs = {
1522         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1523         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]),
1524         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0])
1525     };
1526     verifyScanNoEarlyOut(s, 1, 3);
1527     verifyScanFull(s, kvs);
1528 
1529     // In order to get expected behavior without limiting to group one
1530     // need to wrap SCVFs in SkipFilters
1531     filters = new ArrayList<Filter>();
1532     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1533         CompareOp.EQUAL, VALUES[0])));
1534     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
1535         CompareOp.EQUAL, VALUES[1])));
1536     f = new FilterList(Operator.MUST_PASS_ALL, filters);
1537     s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
1538     s.addFamily(FAMILIES[0]);
1539     s.setFilter(f);
1540     // Expect same KVs
1541     verifyScanNoEarlyOut(s, 1, 3);
1542     verifyScanFull(s, kvs);
1543 
1544     // More tests from HBASE-1821 for Clint and filterIfMissing flag
1545 
1546     byte [][] ROWS_THREE = {
1547         Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"),
1548         Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3")
1549     };
1550 
1551     // Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1])
1552     // Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1])
1553 
1554     KeyValue [] srcKVs = new KeyValue [] {
1555         new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1556         new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
1557         new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]),
1558         new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1])
1559     };
1560 
1561     for(KeyValue kv : srcKVs) {
1562       Put put = new Put(kv.getRow()).add(kv);
1563       put.setDurability(Durability.SKIP_WAL);
1564       this.region.put(put);
1565     }
1566 
1567     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false
1568     // Expect 3 rows (0, 2, 3)
1569     SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILIES[0],
1570         QUALIFIERS_ONE[0], CompareOp.EQUAL, VALUES[0]);
1571     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1572     s.addFamily(FAMILIES[0]);
1573     s.setFilter(scvf);
1574     kvs = new KeyValue [] { srcKVs[0], srcKVs[2], srcKVs[3] };
1575     verifyScanFull(s, kvs);
1576 
1577     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true
1578     // Expect 1 row (0)
1579     scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1580         CompareOp.EQUAL, VALUES[0]);
1581     scvf.setFilterIfMissing(true);
1582     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1583     s.addFamily(FAMILIES[0]);
1584     s.setFilter(scvf);
1585     kvs = new KeyValue [] { srcKVs[0] };
1586     verifyScanFull(s, kvs);
1587 
1588     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1589     // Expect 1 row (3)
1590     scvf = new SingleColumnValueFilter(FAMILIES[0],
1591         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1592     scvf.setFilterIfMissing(true);
1593     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1594     s.addFamily(FAMILIES[0]);
1595     s.setFilter(scvf);
1596     kvs = new KeyValue [] { srcKVs[3] };
1597     verifyScanFull(s, kvs);
1598 
1599     // Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0]
1600     KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]);
1601     this.region.put(new Put(kvA.getRow()).add(kvA));
1602 
1603     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1604     // Expect 1 row (3)
1605     scvf = new SingleColumnValueFilter(FAMILIES[0],
1606         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1607     scvf.setFilterIfMissing(true);
1608     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1609     s.addFamily(FAMILIES[0]);
1610     s.setFilter(scvf);
1611     kvs = new KeyValue [] { srcKVs[3] };
1612     verifyScanFull(s, kvs);
1613 
1614   }
1615 
1616   private void verifyScan(Scan s, long expectedRows, long expectedKeys)
1617   throws IOException {
1618     InternalScanner scanner = this.region.getScanner(s);
1619     List<Cell> results = new ArrayList<Cell>();
1620     int i = 0;
1621     for (boolean done = true; done; i++) {
1622       done = scanner.next(results);
1623       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1624           KeyValue.COMPARATOR);
1625       LOG.info("counter=" + i + ", " + results);
1626       if (results.isEmpty()) break;
1627       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1628           " total but already scanned " + (i+1), expectedRows > i);
1629       assertEquals("Expected " + expectedKeys + " keys per row but " +
1630           "returned " + results.size(), expectedKeys, results.size());
1631       results.clear();
1632     }
1633     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1634         " rows", expectedRows, i);
1635   }
1636 
1637   private void verifyScanNoEarlyOut(Scan s, long expectedRows,
1638       long expectedKeys)
1639   throws IOException {
1640     InternalScanner scanner = this.region.getScanner(s);
1641     List<Cell> results = new ArrayList<Cell>();
1642     int i = 0;
1643     for (boolean done = true; done; i++) {
1644       done = scanner.next(results);
1645       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1646           KeyValue.COMPARATOR);
1647       LOG.info("counter=" + i + ", " + results);
1648       if(results.isEmpty()) break;
1649       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1650           " total but already scanned " + (i+1), expectedRows > i);
1651       assertEquals("Expected " + expectedKeys + " keys per row but " +
1652           "returned " + results.size(), expectedKeys, results.size());
1653       results.clear();
1654     }
1655     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1656         " rows", expectedRows, i);
1657   }
1658 
1659   private void verifyScanFull(Scan s, KeyValue [] kvs)
1660   throws IOException {
1661     InternalScanner scanner = this.region.getScanner(s);
1662     List<Cell> results = new ArrayList<Cell>();
1663     int row = 0;
1664     int idx = 0;
1665     for (boolean done = true; done; row++) {
1666       done = scanner.next(results);
1667       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1668           KeyValue.COMPARATOR);
1669       if(results.isEmpty()) break;
1670       assertTrue("Scanned too many keys! Only expected " + kvs.length +
1671           " total but already scanned " + (results.size() + idx) +
1672           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
1673           kvs.length >= idx + results.size());
1674       for (Cell kv : results) {
1675         LOG.info("row=" + row + ", result=" + kv.toString() +
1676             ", match=" + kvs[idx].toString());
1677         assertTrue("Row mismatch", CellUtil.matchingRow(kv, kvs[idx]));
1678         assertTrue("Family mismatch", CellUtil.matchingFamily(kv, kvs[idx]));
1679         assertTrue("Qualifier mismatch", CellUtil.matchingQualifier(kv, kvs[idx]));
1680         assertTrue("Value mismatch", CellUtil.matchingValue(kv, kvs[idx]));
1681         idx++;
1682       }
1683       results.clear();
1684     }
1685     LOG.info("Looked at " + row + " rows with " + idx + " keys");
1686     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
1687         kvs.length, idx);
1688   }
1689 
1690   private void verifyScanFullNoValues(Scan s, KeyValue [] kvs, boolean useLen)
1691   throws IOException {
1692     InternalScanner scanner = this.region.getScanner(s);
1693     List<Cell> results = new ArrayList<Cell>();
1694     int row = 0;
1695     int idx = 0;
1696     for (boolean more = true; more; row++) {
1697       more = scanner.next(results);
1698       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1699           KeyValue.COMPARATOR);
1700       if(results.isEmpty()) break;
1701       assertTrue("Scanned too many keys! Only expected " + kvs.length +
1702           " total but already scanned " + (results.size() + idx) +
1703           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
1704           kvs.length >= idx + results.size());
1705       for(Cell kv : results) {
1706         LOG.info("row=" + row + ", result=" + kv.toString() +
1707             ", match=" + kvs[idx].toString());
1708 
1709         assertTrue("Row mismatch", CellUtil.matchingRow(kv, kvs[idx]));
1710         assertTrue("Family mismatch", CellUtil.matchingFamily(kv, kvs[idx]));
1711         assertTrue("Qualifier mismatch", CellUtil.matchingQualifier(kv, kvs[idx]));
1712         assertFalse("Should not have returned whole value", CellUtil.matchingValue(kv, kvs[idx]));
1713         if (useLen) {
1714           assertEquals("Value in result is not SIZEOF_INT",
1715                      kv.getValueLength(), Bytes.SIZEOF_INT);
1716           LOG.info("idx = "  + idx + ", len=" + kvs[idx].getValueLength()
1717               + ", actual=" +  Bytes.toInt(CellUtil.cloneValue(kv)));
1718           assertEquals("Scan value should be the length of the actual value. ",
1719                      kvs[idx].getValueLength(), Bytes.toInt(CellUtil.cloneValue(kv)) );
1720           LOG.info("good");
1721         } else {
1722           assertEquals("Value in result is not empty", kv.getValueLength(), 0);
1723         }
1724         idx++;
1725       }
1726       results.clear();
1727     }
1728     LOG.info("Looked at " + row + " rows with " + idx + " keys");
1729     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
1730         kvs.length, idx);
1731   }
1732 
1733   public void testColumnPaginationFilterColumnOffset() throws Exception {
1734     KeyValue [] expectedKVs = {
1735       // testRowOne-0
1736       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1737       // testRowOne-2
1738       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1739       // testRowOne-3
1740       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1741       // testRowTwo-0
1742       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1743       // testRowTwo-2
1744       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1745       // testRowTwo-3
1746       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1747     };
1748     KeyValue [] expectedKVs1 = {
1749       // testRowTwo-0
1750       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1751       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1752       // testRowTwo-2
1753       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1754       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1755       // testRowTwo-3
1756       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1757       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1])
1758     };
1759     KeyValue [] expectedKVs2 = {
1760       // testRowTwo-0
1761       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1762       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1763       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1764       // testRowTwo-2
1765       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1766       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1767       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1768       // testRowTwo-3
1769       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1770       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1771       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1])
1772     };
1773     KeyValue [] expectedKVs3 = {
1774       // testRowTwo-0
1775       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1776       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1777       // testRowTwo-2
1778       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1779       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1780       // testRowTwo-3
1781       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1782       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1783     };
1784     Scan s = new Scan();
1785 
1786     // Page size 1.
1787     long expectedRows = 6;
1788     long expectedKeys = 1;
1789     s.setFilter(new ColumnPaginationFilter(1, QUALIFIERS_ONE[1]));
1790     verifyScan(s, expectedRows, expectedKeys);
1791     this.verifyScanFull(s, expectedKVs);
1792 
1793     // Page size 2.
1794     expectedRows = 3;
1795     expectedKeys = 2;
1796     s = new Scan();
1797     s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
1798     verifyScan(s, expectedRows, expectedKeys);
1799     this.verifyScanFull(s, expectedKVs1);
1800 
1801     // Page size 3 across multiple column families.
1802     expectedRows = 3;
1803     expectedKeys = 3;
1804     s = new Scan();
1805     s.setFilter(new ColumnPaginationFilter(3, QUALIFIERS_TWO[2]));
1806     verifyScan(s, expectedRows, expectedKeys);
1807     this.verifyScanFull(s, expectedKVs2);
1808 
1809     // Page size 2 restricted to one column family.
1810     expectedRows = 3;
1811     expectedKeys = 2;
1812     s = new Scan();
1813     s.addFamily(FAMILIES[1]);
1814     s.setFilter(new ColumnPaginationFilter(2, QUALIFIERS_TWO[2]));
1815     this.verifyScanFull(s, expectedKVs3);
1816   }
1817 
1818   @Test
1819   public void testColumnPaginationFilter() throws Exception {
1820       // Test that the filter skips multiple column versions.
1821       Put p = new Put(ROWS_ONE[0]);
1822       p.setDurability(Durability.SKIP_WAL);
1823       p.add(FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]);
1824       this.region.put(p);
1825       this.region.flush(true);
1826 
1827       // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
1828       KeyValue [] expectedKVs = {
1829         // testRowOne-0
1830         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1831         // testRowOne-2
1832         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1833         // testRowOne-3
1834         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1835         // testRowTwo-0
1836         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1837         // testRowTwo-2
1838         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1839         // testRowTwo-3
1840         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
1841       };
1842 
1843       // Set of KVs (page: 3; pageSize: 1)  - the third set of 1 column per row
1844       KeyValue [] expectedKVs2 = {
1845         // testRowOne-0
1846         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1847         // testRowOne-2
1848         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1849         // testRowOne-3
1850         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1851         // testRowTwo-0
1852         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1853         // testRowTwo-2
1854         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1855         // testRowTwo-3
1856         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1857       };
1858 
1859       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1860       KeyValue [] expectedKVs3 = {
1861         // testRowOne-0
1862         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1863         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1864         // testRowOne-2
1865         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1866         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1867         // testRowOne-3
1868         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1869         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1870         // testRowTwo-0
1871         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1872         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1873         // testRowTwo-2
1874         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1875         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1876         // testRowTwo-3
1877         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1878         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1879       };
1880 
1881 
1882       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1883       KeyValue [] expectedKVs4 = {
1884 
1885       };
1886 
1887       long expectedRows = this.numRows;
1888       long expectedKeys = 1;
1889       Scan s = new Scan();
1890 
1891 
1892       // Page 1; 1 Column per page  (Limit 1, Offset 0)
1893       s.setFilter(new ColumnPaginationFilter(1,0));
1894       verifyScan(s, expectedRows, expectedKeys);
1895       this.verifyScanFull(s, expectedKVs);
1896 
1897       // Page 3; 1 Result per page  (Limit 1, Offset 2)
1898       s.setFilter(new ColumnPaginationFilter(1,2));
1899       verifyScan(s, expectedRows, expectedKeys);
1900       this.verifyScanFull(s, expectedKVs2);
1901 
1902       // Page 2; 2 Results per page (Limit 2, Offset 2)
1903       s.setFilter(new ColumnPaginationFilter(2,2));
1904       expectedKeys = 2;
1905       verifyScan(s, expectedRows, expectedKeys);
1906       this.verifyScanFull(s, expectedKVs3);
1907 
1908       // Page 8; 20 Results per page (no results) (Limit 20, Offset 140)
1909       s.setFilter(new ColumnPaginationFilter(20,140));
1910       expectedKeys = 0;
1911       expectedRows = 0;
1912       verifyScan(s, expectedRows, 0);
1913       this.verifyScanFull(s, expectedKVs4);
1914   }
1915 
1916   @Test
1917   public void testKeyOnlyFilter() throws Exception {
1918 
1919     // KVs in first 6 rows
1920     KeyValue [] expectedKVs = {
1921       // testRowOne-0
1922       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1923       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1924       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1925       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1926       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1927       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1928       // testRowOne-2
1929       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1930       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1931       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1932       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1933       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1934       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1935       // testRowOne-3
1936       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1937       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1938       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1939       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1940       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1941       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1942       // testRowTwo-0
1943       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1944       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1945       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1946       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1947       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1948       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1949       // testRowTwo-2
1950       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1951       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1952       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1953       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1954       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1955       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1956       // testRowTwo-3
1957       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1958       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1959       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1960       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1961       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1962       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
1963     };
1964 
1965     // Grab all 6 rows
1966     long expectedRows = 6;
1967     long expectedKeys = this.colsPerRow;
1968     for (boolean useLen : new boolean[]{false,true}) {
1969       Scan s = new Scan();
1970       s.setFilter(new KeyOnlyFilter(useLen));
1971       verifyScan(s, expectedRows, expectedKeys);
1972       verifyScanFullNoValues(s, expectedKVs, useLen);
1973     }
1974   }
1975   
1976   /**
1977    * Filter which makes sleeps for a second between each row of a scan.
1978    * This can be useful for manual testing of bugs like HBASE-5973. For example:
1979    * <code>
1980    * create 't1', 'f1'
1981    * 1.upto(100)  { |x| put 't1', 'r' + x.to_s, 'f1:q1', 'hi' }
1982    * import org.apache.hadoop.hbase.filter.TestFilter
1983    * scan 't1', { FILTER => TestFilter::SlowScanFilter.new(), CACHE => 50 }
1984    * </code>
1985    */
1986   public static class SlowScanFilter extends FilterBase {
1987     private static Thread ipcHandlerThread = null;
1988     
1989     @Override
1990     public byte [] toByteArray() {return null;}
1991 
1992     @Override
1993     public ReturnCode filterKeyValue(Cell ignored) throws IOException {
1994       return ReturnCode.INCLUDE;
1995     }
1996 
1997     @Override
1998     public boolean filterRow() throws IOException {
1999       ipcHandlerThread = Thread.currentThread();
2000       try {
2001         LOG.info("Handler thread " + ipcHandlerThread + " sleeping in filter...");
2002         Thread.sleep(1000);
2003       } catch (InterruptedException e) {
2004         Throwables.propagate(e);
2005       }
2006       return super.filterRow();
2007     }
2008   }
2009 
2010   public void testNestedFilterListWithSCVF() throws IOException {
2011     byte[] columnStatus = Bytes.toBytes("S");
2012     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testNestedFilterListWithSCVF"));
2013     htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
2014     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
2015     HRegion testRegion = HRegion.createHRegion(info, TEST_UTIL.getDataTestDir(),
2016       TEST_UTIL.getConfiguration(), htd);
2017     for(int i=0; i<10; i++) {
2018       Put p = new Put(Bytes.toBytes("row" + i));
2019       p.setDurability(Durability.SKIP_WAL);
2020       p.add(FAMILIES[0], columnStatus, Bytes.toBytes(i%2));
2021       testRegion.put(p);
2022     }
2023     testRegion.flush(true);
2024     // 1. got rows > "row4"
2025     Filter rowFilter = new RowFilter(CompareOp.GREATER,new BinaryComparator(Bytes.toBytes("row4")));
2026     Scan s1 = new Scan();
2027     s1.setFilter(rowFilter);
2028     InternalScanner scanner = testRegion.getScanner(s1);
2029     List<Cell> results = new ArrayList<Cell>();
2030     int i = 5;
2031     for (boolean done = true; done; i++) {
2032       done = scanner.next(results);
2033       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2034       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2035       results.clear();
2036     }
2037     // 2. got rows <= "row4" and S=
2038     FilterList subFilterList = new FilterList(FilterList.Operator.MUST_PASS_ALL);
2039     Filter subFilter1 = new RowFilter(CompareOp.LESS_OR_EQUAL,
2040       new BinaryComparator(Bytes.toBytes("row4")));
2041     subFilterList.addFilter(subFilter1);
2042     Filter subFilter2 = new SingleColumnValueFilter(FAMILIES[0], columnStatus, CompareOp.EQUAL,
2043       Bytes.toBytes(0));
2044     subFilterList.addFilter(subFilter2);
2045     s1 = new Scan();
2046     s1.setFilter(subFilterList);
2047     scanner = testRegion.getScanner(s1);
2048     results = new ArrayList<Cell>();
2049     for (i=0; i<=4; i+=2) {
2050       scanner.next(results);
2051       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2052       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2053       results.clear();
2054     }
2055     assertFalse(scanner.next(results));
2056     // 3. let's begin to verify nested filter list
2057     // 3.1 add rowFilter, then add subFilterList
2058     FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
2059     filterList.addFilter(rowFilter);
2060     filterList.addFilter(subFilterList);
2061     s1 = new Scan();
2062     s1.setFilter(filterList);
2063     scanner = testRegion.getScanner(s1);
2064     results = new ArrayList<Cell>();
2065     for (i=0; i<=4; i+=2) {
2066       scanner.next(results);
2067       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2068       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2069       results.clear();
2070     }
2071     for (i=5; i<=9; i++) {
2072       scanner.next(results);
2073       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2074       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2075       results.clear();
2076     }
2077     assertFalse(scanner.next(results));
2078     // 3.2 MAGIC here! add subFilterList first, then add rowFilter
2079     filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
2080     filterList.addFilter(subFilterList);
2081     filterList.addFilter(rowFilter);
2082     s1 = new Scan();
2083     s1.setFilter(filterList);
2084     scanner = testRegion.getScanner(s1);
2085     results = new ArrayList<Cell>();
2086     for (i=0; i<=4; i+=2) {
2087       scanner.next(results);
2088       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2089       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2090       results.clear();
2091     }
2092     for (i=5; i<=9; i++) {
2093       scanner.next(results);
2094       assertTrue(CellUtil.matchingRow(results.get(0), Bytes.toBytes("row" + i)));
2095       assertEquals(Bytes.toInt(CellUtil.cloneValue(results.get(0))), i%2);
2096       results.clear();
2097     }
2098     assertFalse(scanner.next(results));
2099     WAL wal = ((HRegion)testRegion).getWAL();
2100     ((HRegion)testRegion).close();
2101     wal.close();
2102   }      
2103 }