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