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