View Javadoc

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