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