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