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