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.IOException;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.List;
27  import java.util.concurrent.atomic.AtomicInteger;
28  
29  import junit.framework.Assert;
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.hbase.HBaseTestCase;
33  import org.apache.hadoop.hbase.HColumnDescriptor;
34  import org.apache.hadoop.hbase.HConstants;
35  import org.apache.hadoop.hbase.HRegionInfo;
36  import org.apache.hadoop.hbase.HTableDescriptor;
37  import org.apache.hadoop.hbase.KeyValue;
38  import org.apache.hadoop.hbase.client.Delete;
39  import org.apache.hadoop.hbase.client.Put;
40  import org.apache.hadoop.hbase.client.Scan;
41  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
42  import org.apache.hadoop.hbase.filter.FilterList.Operator;
43  import org.apache.hadoop.hbase.regionserver.HRegion;
44  import org.apache.hadoop.hbase.regionserver.InternalScanner;
45  import org.apache.hadoop.hbase.util.Bytes;
46  
47  /**
48   * Test filters at the HRegion doorstep.
49   */
50  public class TestFilter extends HBaseTestCase {
51    private final Log LOG = LogFactory.getLog(this.getClass());
52    private HRegion region;
53  
54    //
55    // Rows, Qualifiers, and Values are in two groups, One and Two.
56    //
57  
58    private static final byte [][] ROWS_ONE = {
59        Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
60        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3")
61    };
62  
63    private static final byte [][] ROWS_TWO = {
64        Bytes.toBytes("testRowTwo-0"), Bytes.toBytes("testRowTwo-1"),
65        Bytes.toBytes("testRowTwo-2"), Bytes.toBytes("testRowTwo-3")
66    };
67  
68    private static final byte [][] FAMILIES = {
69      Bytes.toBytes("testFamilyOne"), Bytes.toBytes("testFamilyTwo")
70    };
71  
72    private static final byte [][] QUALIFIERS_ONE = {
73      Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"),
74      Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3")
75    };
76  
77    private static final byte [][] QUALIFIERS_TWO = {
78      Bytes.toBytes("testQualifierTwo-0"), Bytes.toBytes("testQualifierTwo-1"),
79      Bytes.toBytes("testQualifierTwo-2"), Bytes.toBytes("testQualifierTwo-3")
80    };
81  
82    private static final byte [][] VALUES = {
83      Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo")
84    };
85  
86    private long numRows = ROWS_ONE.length + ROWS_TWO.length;
87    private long colsPerRow = FAMILIES.length * QUALIFIERS_ONE.length;
88  
89  
90    protected void setUp() throws Exception {
91      super.setUp();
92      HTableDescriptor htd = new HTableDescriptor(getName());
93      htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
94      htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
95      HRegionInfo info = new HRegionInfo(htd, null, null, false);
96      this.region = HRegion.createHRegion(info, this.testDir, this.conf);
97  
98      // Insert first half
99      for(byte [] ROW : ROWS_ONE) {
100       Put p = new Put(ROW);
101       for(byte [] QUALIFIER : QUALIFIERS_ONE) {
102         p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
103       }
104       this.region.put(p);
105     }
106     for(byte [] ROW : ROWS_TWO) {
107       Put p = new Put(ROW);
108       for(byte [] QUALIFIER : QUALIFIERS_TWO) {
109         p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
110       }
111       this.region.put(p);
112     }
113 
114     // Flush
115     this.region.flushcache();
116 
117     // Insert second half (reverse families)
118     for(byte [] ROW : ROWS_ONE) {
119       Put p = new Put(ROW);
120       for(byte [] QUALIFIER : QUALIFIERS_ONE) {
121         p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
122       }
123       this.region.put(p);
124     }
125     for(byte [] ROW : ROWS_TWO) {
126       Put p = new Put(ROW);
127       for(byte [] QUALIFIER : QUALIFIERS_TWO) {
128         p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
129       }
130       this.region.put(p);
131     }
132 
133     // Delete the second qualifier from all rows and families
134     for(byte [] ROW : ROWS_ONE) {
135       Delete d = new Delete(ROW);
136       d.deleteColumns(FAMILIES[0], QUALIFIERS_ONE[1]);
137       d.deleteColumns(FAMILIES[1], QUALIFIERS_ONE[1]);
138       this.region.delete(d, null, false);
139     }
140     for(byte [] ROW : ROWS_TWO) {
141       Delete d = new Delete(ROW);
142       d.deleteColumns(FAMILIES[0], QUALIFIERS_TWO[1]);
143       d.deleteColumns(FAMILIES[1], QUALIFIERS_TWO[1]);
144       this.region.delete(d, null, false);
145     }
146     colsPerRow -= 2;
147 
148     // Delete the second rows from both groups, one column at a time
149     for(byte [] QUALIFIER : QUALIFIERS_ONE) {
150       Delete d = new Delete(ROWS_ONE[1]);
151       d.deleteColumns(FAMILIES[0], QUALIFIER);
152       d.deleteColumns(FAMILIES[1], QUALIFIER);
153       this.region.delete(d, null, false);
154     }
155     for(byte [] QUALIFIER : QUALIFIERS_TWO) {
156       Delete d = new Delete(ROWS_TWO[1]);
157       d.deleteColumns(FAMILIES[0], QUALIFIER);
158       d.deleteColumns(FAMILIES[1], QUALIFIER);
159       this.region.delete(d, null, false);
160     }
161     numRows -= 2;
162   }
163 
164   protected void tearDown() throws Exception {
165     this.region.close();
166     super.tearDown();
167   }
168 
169   public void testNoFilter() throws Exception {
170     // No filter
171     long expectedRows = this.numRows;
172     long expectedKeys = this.colsPerRow;
173 
174     // Both families
175     Scan s = new Scan();
176     verifyScan(s, expectedRows, expectedKeys);
177 
178     // One family
179     s = new Scan();
180     s.addFamily(FAMILIES[0]);
181     verifyScan(s, expectedRows, expectedKeys/2);
182   }
183 
184   public void testPrefixFilter() throws Exception {
185     // Grab rows from group one (half of total)
186     long expectedRows = this.numRows / 2;
187     long expectedKeys = this.colsPerRow;
188     Scan s = new Scan();
189     s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
190     verifyScan(s, expectedRows, expectedKeys);
191   }
192 
193   public void testPageFilter() throws Exception {
194 
195     // KVs in first 6 rows
196     KeyValue [] expectedKVs = {
197       // testRowOne-0
198       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
199       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
200       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
201       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
202       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
203       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
204       // testRowOne-2
205       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
206       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
207       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
208       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
209       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
210       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
211       // testRowOne-3
212       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
213       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
214       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
215       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
216       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
217       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
218       // testRowTwo-0
219       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
220       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
221       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
222       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
223       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
224       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
225       // testRowTwo-2
226       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
227       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
228       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
229       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
230       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
231       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
232       // testRowTwo-3
233       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
234       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
235       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
236       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
237       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
238       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
239     };
240 
241     // Grab all 6 rows
242     long expectedRows = 6;
243     long expectedKeys = this.colsPerRow;
244     Scan s = new Scan();
245     s.setFilter(new PageFilter(expectedRows));
246     verifyScan(s, expectedRows, expectedKeys);
247     s.setFilter(new PageFilter(expectedRows));
248     verifyScanFull(s, expectedKVs);
249 
250     // Grab first 4 rows (6 cols per row)
251     expectedRows = 4;
252     expectedKeys = this.colsPerRow;
253     s = new Scan();
254     s.setFilter(new PageFilter(expectedRows));
255     verifyScan(s, expectedRows, expectedKeys);
256     s.setFilter(new PageFilter(expectedRows));
257     verifyScanFull(s, Arrays.copyOf(expectedKVs, 24));
258 
259     // Grab first 2 rows
260     expectedRows = 2;
261     expectedKeys = this.colsPerRow;
262     s = new Scan();
263     s.setFilter(new PageFilter(expectedRows));
264     verifyScan(s, expectedRows, expectedKeys);
265     s.setFilter(new PageFilter(expectedRows));
266     verifyScanFull(s, Arrays.copyOf(expectedKVs, 12));
267 
268     // Grab first row
269     expectedRows = 1;
270     expectedKeys = this.colsPerRow;
271     s = new Scan();
272     s.setFilter(new PageFilter(expectedRows));
273     verifyScan(s, expectedRows, expectedKeys);
274     s.setFilter(new PageFilter(expectedRows));
275     verifyScanFull(s, Arrays.copyOf(expectedKVs, 6));
276 
277   }
278 
279   /**
280    * Tests the the {@link WhileMatchFilter} works in combination with a
281    * {@link Filter} that uses the
282    * {@link Filter#filterRow()} method.
283    *
284    * See HBASE-2258.
285    *
286    * @throws Exception
287    */
288   public void testWhileMatchFilterWithFilterRow() throws Exception {
289     final int pageSize = 4;
290 
291     Scan s = new Scan();
292     WhileMatchFilter filter = new WhileMatchFilter(new PageFilter(pageSize));
293     s.setFilter(filter);
294 
295     InternalScanner scanner = this.region.getScanner(s);
296     int scannerCounter = 0;
297     while (true) {
298       boolean isMoreResults = scanner.next(new ArrayList<KeyValue>());
299       scannerCounter++;
300 
301       if (scannerCounter >= pageSize) {
302         Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
303       }
304       if (!isMoreResults) {
305         break;
306       }
307     }
308     Assert.assertEquals("The page filter returned more rows than expected", pageSize, scannerCounter);
309   }
310 
311   /**
312    * Tests the the {@link WhileMatchFilter} works in combination with a
313    * {@link Filter} that uses the
314    * {@link Filter#filterRowKey(byte[], int, int)} method.
315    *
316    * See HBASE-2258.
317    *
318    * @throws Exception
319    */
320   public void testWhileMatchFilterWithFilterRowKey() throws Exception {
321     Scan s = new Scan();
322     String prefix = "testRowOne";
323     WhileMatchFilter filter = new WhileMatchFilter(new PrefixFilter(Bytes.toBytes(prefix)));
324     s.setFilter(filter);
325 
326     InternalScanner scanner = this.region.getScanner(s);
327     while (true) {
328       ArrayList<KeyValue> values = new ArrayList<KeyValue>();
329       boolean isMoreResults = scanner.next(values);
330       if (!isMoreResults || !Bytes.toString(values.get(0).getRow()).startsWith(prefix)) {
331         Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
332       }
333       if (!isMoreResults) {
334         break;
335       }
336     }
337   }
338 
339   /**
340    * Tests the the {@link WhileMatchFilter} works in combination with a
341    * {@link Filter} that uses the
342    * {@link Filter#filterKeyValue(org.apache.hadoop.hbase.KeyValue)} method.
343    *
344    * See HBASE-2258.
345    *
346    * @throws Exception
347    */
348   public void testWhileMatchFilterWithFilterKeyValue() throws Exception {
349     Scan s = new Scan();
350     WhileMatchFilter filter = new WhileMatchFilter(
351         new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0], CompareOp.EQUAL, Bytes.toBytes("foo"))
352     );
353     s.setFilter(filter);
354 
355     InternalScanner scanner = this.region.getScanner(s);
356     while (true) {
357       ArrayList<KeyValue> values = new ArrayList<KeyValue>();
358       boolean isMoreResults = scanner.next(values);
359       Assert.assertTrue("The WhileMatchFilter should now filter all remaining", filter.filterAllRemaining());
360       if (!isMoreResults) {
361         break;
362       }
363     }
364   }
365 
366   public void testInclusiveStopFilter() throws IOException {
367 
368     // Grab rows from group one
369 
370     // If we just use start/stop row, we get total/2 - 1 rows
371     long expectedRows = (this.numRows / 2) - 1;
372     long expectedKeys = this.colsPerRow;
373     Scan s = new Scan(Bytes.toBytes("testRowOne-0"),
374         Bytes.toBytes("testRowOne-3"));
375     verifyScan(s, expectedRows, expectedKeys);
376 
377     // Now use start row with inclusive stop filter
378     expectedRows = this.numRows / 2;
379     s = new Scan(Bytes.toBytes("testRowOne-0"));
380     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-3")));
381     verifyScan(s, expectedRows, expectedKeys);
382 
383     // Grab rows from group two
384 
385     // If we just use start/stop row, we get total/2 - 1 rows
386     expectedRows = (this.numRows / 2) - 1;
387     expectedKeys = this.colsPerRow;
388     s = new Scan(Bytes.toBytes("testRowTwo-0"),
389         Bytes.toBytes("testRowTwo-3"));
390     verifyScan(s, expectedRows, expectedKeys);
391 
392     // Now use start row with inclusive stop filter
393     expectedRows = this.numRows / 2;
394     s = new Scan(Bytes.toBytes("testRowTwo-0"));
395     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-3")));
396     verifyScan(s, expectedRows, expectedKeys);
397 
398   }
399 
400   public void testQualifierFilter() throws IOException {
401 
402     // Match two keys (one from each family) in half the rows
403     long expectedRows = this.numRows / 2;
404     long expectedKeys = 2;
405     Filter f = new QualifierFilter(CompareOp.EQUAL,
406         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
407     Scan s = new Scan();
408     s.setFilter(f);
409     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
410 
411     // Match keys less than same qualifier
412     // Expect only two keys (one from each family) in half the rows
413     expectedRows = this.numRows / 2;
414     expectedKeys = 2;
415     f = new QualifierFilter(CompareOp.LESS,
416         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
417     s = new Scan();
418     s.setFilter(f);
419     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
420 
421     // Match keys less than or equal
422     // Expect four keys (two from each family) in half the rows
423     expectedRows = this.numRows / 2;
424     expectedKeys = 4;
425     f = new QualifierFilter(CompareOp.LESS_OR_EQUAL,
426         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
427     s = new Scan();
428     s.setFilter(f);
429     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
430 
431     // Match keys not equal
432     // Expect four keys (two from each family)
433     // Only look in first group of rows
434     expectedRows = this.numRows / 2;
435     expectedKeys = 4;
436     f = new QualifierFilter(CompareOp.NOT_EQUAL,
437         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
438     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
439     s.setFilter(f);
440     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
441 
442     // Match keys greater or equal
443     // Expect four keys (two from each family)
444     // Only look in first group of rows
445     expectedRows = this.numRows / 2;
446     expectedKeys = 4;
447     f = new QualifierFilter(CompareOp.GREATER_OR_EQUAL,
448         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
449     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
450     s.setFilter(f);
451     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
452 
453     // Match keys greater
454     // Expect two keys (one from each family)
455     // Only look in first group of rows
456     expectedRows = this.numRows / 2;
457     expectedKeys = 2;
458     f = new QualifierFilter(CompareOp.GREATER,
459         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
460     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
461     s.setFilter(f);
462     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
463 
464     // Match keys not equal to
465     // Look across rows and fully validate the keys and ordering
466     // Expect varied numbers of keys, 4 per row in group one, 6 per row in group two
467     f = new QualifierFilter(CompareOp.NOT_EQUAL,
468         new BinaryComparator(QUALIFIERS_ONE[2]));
469     s = new Scan();
470     s.setFilter(f);
471 
472     KeyValue [] kvs = {
473         // testRowOne-0
474         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
475         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
476         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
477         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
478         // testRowOne-2
479         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
480         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
481         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
482         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
483         // testRowOne-3
484         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
485         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
486         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
487         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
488         // testRowTwo-0
489         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
490         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
491         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
492         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
493         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
494         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
495         // testRowTwo-2
496         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
497         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
498         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
499         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
500         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
501         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
502         // testRowTwo-3
503         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
504         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
505         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
506         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
507         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
508         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
509     };
510     verifyScanFull(s, kvs);
511 
512 
513     // Test across rows and groups with a regex
514     // Filter out "test*-2"
515     // Expect 4 keys per row across both groups
516     f = new QualifierFilter(CompareOp.NOT_EQUAL,
517         new RegexStringComparator("test.+-2"));
518     s = new Scan();
519     s.setFilter(f);
520 
521     kvs = new KeyValue [] {
522         // testRowOne-0
523         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
524         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
525         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
526         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
527         // testRowOne-2
528         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
529         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
530         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
531         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
532         // testRowOne-3
533         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
534         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
535         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
536         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
537         // testRowTwo-0
538         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
539         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
540         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
541         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
542         // testRowTwo-2
543         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
544         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
545         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
546         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
547         // testRowTwo-3
548         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
549         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
550         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
551         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
552     };
553     verifyScanFull(s, kvs);
554 
555   }
556 
557   public void testRowFilter() throws IOException {
558 
559     // Match a single row, all keys
560     long expectedRows = 1;
561     long expectedKeys = this.colsPerRow;
562     Filter f = new RowFilter(CompareOp.EQUAL,
563         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
564     Scan s = new Scan();
565     s.setFilter(f);
566     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
567 
568     // Match a two rows, one from each group, using regex
569     expectedRows = 2;
570     expectedKeys = this.colsPerRow;
571     f = new RowFilter(CompareOp.EQUAL,
572         new RegexStringComparator("testRow.+-2"));
573     s = new Scan();
574     s.setFilter(f);
575     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
576 
577     // Match rows less than
578     // Expect all keys in one row
579     expectedRows = 1;
580     expectedKeys = this.colsPerRow;
581     f = new RowFilter(CompareOp.LESS,
582         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
583     s = new Scan();
584     s.setFilter(f);
585     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
586 
587     // Match rows less than or equal
588     // Expect all keys in two rows
589     expectedRows = 2;
590     expectedKeys = this.colsPerRow;
591     f = new RowFilter(CompareOp.LESS_OR_EQUAL,
592         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
593     s = new Scan();
594     s.setFilter(f);
595     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
596 
597     // Match rows not equal
598     // Expect all keys in all but one row
599     expectedRows = this.numRows - 1;
600     expectedKeys = this.colsPerRow;
601     f = new RowFilter(CompareOp.NOT_EQUAL,
602         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
603     s = new Scan();
604     s.setFilter(f);
605     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
606 
607     // Match keys greater or equal
608     // Expect all keys in all but one row
609     expectedRows = this.numRows - 1;
610     expectedKeys = this.colsPerRow;
611     f = new RowFilter(CompareOp.GREATER_OR_EQUAL,
612         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
613     s = new Scan();
614     s.setFilter(f);
615     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
616 
617     // Match keys greater
618     // Expect all keys in all but two rows
619     expectedRows = this.numRows - 2;
620     expectedKeys = this.colsPerRow;
621     f = new RowFilter(CompareOp.GREATER,
622         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
623     s = new Scan();
624     s.setFilter(f);
625     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
626 
627     // Match rows not equal to testRowTwo-2
628     // Look across rows and fully validate the keys and ordering
629     // Should see all keys in all rows but testRowTwo-2
630     f = new RowFilter(CompareOp.NOT_EQUAL,
631         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
632     s = new Scan();
633     s.setFilter(f);
634 
635     KeyValue [] kvs = {
636         // testRowOne-0
637         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
638         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
639         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
640         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
641         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
642         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
643         // testRowOne-3
644         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
645         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
646         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
647         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
648         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
649         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
650         // testRowTwo-0
651         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
652         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
653         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
654         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
655         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
656         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
657         // testRowTwo-2
658         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
659         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
660         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
661         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
662         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
663         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
664         // testRowTwo-3
665         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
666         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
667         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
668         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
669         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
670         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
671     };
672     verifyScanFull(s, kvs);
673 
674 
675     // Test across rows and groups with a regex
676     // Filter out everything that doesn't match "*-2"
677     // Expect all keys in two rows
678     f = new RowFilter(CompareOp.EQUAL,
679         new RegexStringComparator(".+-2"));
680     s = new Scan();
681     s.setFilter(f);
682 
683     kvs = new KeyValue [] {
684         // testRowOne-2
685         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
686         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
687         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
688         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
689         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
690         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
691         // testRowTwo-2
692         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
693         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
694         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
695         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
696         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
697         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
698     };
699     verifyScanFull(s, kvs);
700 
701   }
702 
703   public void testValueFilter() throws IOException {
704 
705     // Match group one rows
706     long expectedRows = this.numRows / 2;
707     long expectedKeys = this.colsPerRow;
708     Filter f = new ValueFilter(CompareOp.EQUAL,
709         new BinaryComparator(Bytes.toBytes("testValueOne")));
710     Scan s = new Scan();
711     s.setFilter(f);
712     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
713 
714     // Match group two rows
715     expectedRows = this.numRows / 2;
716     expectedKeys = this.colsPerRow;
717     f = new ValueFilter(CompareOp.EQUAL,
718         new BinaryComparator(Bytes.toBytes("testValueTwo")));
719     s = new Scan();
720     s.setFilter(f);
721     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
722 
723     // Match all values using regex
724     expectedRows = this.numRows;
725     expectedKeys = this.colsPerRow;
726     f = new ValueFilter(CompareOp.EQUAL,
727         new RegexStringComparator("testValue((One)|(Two))"));
728     s = new Scan();
729     s.setFilter(f);
730     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
731 
732     // Match values less than
733     // Expect group one rows
734     expectedRows = this.numRows / 2;
735     expectedKeys = this.colsPerRow;
736     f = new ValueFilter(CompareOp.LESS,
737         new BinaryComparator(Bytes.toBytes("testValueTwo")));
738     s = new Scan();
739     s.setFilter(f);
740     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
741 
742     // Match values less than or equal
743     // Expect all rows
744     expectedRows = this.numRows;
745     expectedKeys = this.colsPerRow;
746     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
747         new BinaryComparator(Bytes.toBytes("testValueTwo")));
748     s = new Scan();
749     s.setFilter(f);
750     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
751 
752     // Match values less than or equal
753     // Expect group one rows
754     expectedRows = this.numRows / 2;
755     expectedKeys = this.colsPerRow;
756     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
757         new BinaryComparator(Bytes.toBytes("testValueOne")));
758     s = new Scan();
759     s.setFilter(f);
760     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
761 
762     // Match values not equal
763     // Expect half the rows
764     expectedRows = this.numRows / 2;
765     expectedKeys = this.colsPerRow;
766     f = new ValueFilter(CompareOp.NOT_EQUAL,
767         new BinaryComparator(Bytes.toBytes("testValueOne")));
768     s = new Scan();
769     s.setFilter(f);
770     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
771 
772     // Match values greater or equal
773     // Expect all rows
774     expectedRows = this.numRows;
775     expectedKeys = this.colsPerRow;
776     f = new ValueFilter(CompareOp.GREATER_OR_EQUAL,
777         new BinaryComparator(Bytes.toBytes("testValueOne")));
778     s = new Scan();
779     s.setFilter(f);
780     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
781 
782     // Match values greater
783     // Expect half rows
784     expectedRows = this.numRows / 2;
785     expectedKeys = this.colsPerRow;
786     f = new ValueFilter(CompareOp.GREATER,
787         new BinaryComparator(Bytes.toBytes("testValueOne")));
788     s = new Scan();
789     s.setFilter(f);
790     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
791 
792     // Match values not equal to testValueOne
793     // Look across rows and fully validate the keys and ordering
794     // Should see all keys in all group two rows
795     f = new ValueFilter(CompareOp.NOT_EQUAL,
796         new BinaryComparator(Bytes.toBytes("testValueOne")));
797     s = new Scan();
798     s.setFilter(f);
799 
800     KeyValue [] kvs = {
801         // testRowTwo-0
802         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
803         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
804         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
805         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
806         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
807         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
808         // testRowTwo-2
809         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
810         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
811         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
812         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
813         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
814         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
815         // testRowTwo-3
816         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
817         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
818         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
819         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
820         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
821         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
822     };
823     verifyScanFull(s, kvs);
824   }
825 
826   public void testSkipFilter() throws IOException {
827 
828     // Test for qualifier regex: "testQualifierOne-2"
829     // Should only get rows from second group, and all keys
830     Filter f = new SkipFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
831         new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))));
832     Scan s = new Scan();
833     s.setFilter(f);
834 
835     KeyValue [] kvs = {
836         // testRowTwo-0
837         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
838         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
839         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
840         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
841         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
842         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
843         // testRowTwo-2
844         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
845         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
846         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
847         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
848         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
849         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
850         // testRowTwo-3
851         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
852         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
853         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
854         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
855         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
856         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
857     };
858     verifyScanFull(s, kvs);
859   }
860 
861   // TODO: This is important... need many more tests for ordering, etc
862   // There are limited tests elsewhere but we need HRegion level ones here
863   public void testFilterList() throws IOException {
864 
865     // Test getting a single row, single key using Row, Qualifier, and Value
866     // regular expression and substring filters
867     // Use must pass all
868     List<Filter> filters = new ArrayList<Filter>();
869     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
870     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
871     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
872     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
873     Scan s = new Scan();
874     s.addFamily(FAMILIES[0]);
875     s.setFilter(f);
876     KeyValue [] kvs = {
877         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0])
878     };
879     verifyScanFull(s, kvs);
880 
881     // Test getting everything with a MUST_PASS_ONE filter including row, qf, val
882     // regular expression and substring filters
883     filters.clear();
884     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+Two.+")));
885     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
886     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
887     f = new FilterList(Operator.MUST_PASS_ONE, filters);
888     s = new Scan();
889     s.setFilter(f);
890     verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow);
891 
892 
893   }
894 
895   public void testFirstKeyOnlyFilter() throws IOException {
896     Scan s = new Scan();
897     s.setFilter(new FirstKeyOnlyFilter());
898     // Expected KVs, the first KV from each of the remaining 6 rows
899     KeyValue [] kvs = {
900         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
901         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
902         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
903         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
904         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
905         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
906     };
907     verifyScanFull(s, kvs);
908   }
909 
910   public void testSingleColumnValueFilter() throws IOException {
911 
912     // From HBASE-1821
913     // Desired action is to combine two SCVF in a FilterList
914     // Want to return only rows that match both conditions
915 
916     // Need to change one of the group one columns to use group two value
917     Put p = new Put(ROWS_ONE[2]);
918     p.add(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]);
919     this.region.put(p);
920 
921     // Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1])
922     // Since group two rows don't have these qualifiers, they will pass
923     // so limiting scan to group one
924     List<Filter> filters = new ArrayList<Filter>();
925     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
926         CompareOp.EQUAL, VALUES[0]));
927     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
928         CompareOp.EQUAL, VALUES[1]));
929     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
930     Scan s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
931     s.addFamily(FAMILIES[0]);
932     s.setFilter(f);
933     // Expect only one row, all qualifiers
934     KeyValue [] kvs = {
935         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
936         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]),
937         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0])
938     };
939     verifyScanNoEarlyOut(s, 1, 3);
940     verifyScanFull(s, kvs);
941 
942     // In order to get expected behavior without limiting to group one
943     // need to wrap SCVFs in SkipFilters
944     filters = new ArrayList<Filter>();
945     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
946         CompareOp.EQUAL, VALUES[0])));
947     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
948         CompareOp.EQUAL, VALUES[1])));
949     f = new FilterList(Operator.MUST_PASS_ALL, filters);
950     s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
951     s.addFamily(FAMILIES[0]);
952     s.setFilter(f);
953     // Expect same KVs
954     verifyScanNoEarlyOut(s, 1, 3);
955     verifyScanFull(s, kvs);
956 
957     // More tests from HBASE-1821 for Clint and filterIfMissing flag
958 
959     byte [][] ROWS_THREE = {
960         Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"),
961         Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3")
962     };
963 
964     // Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1])
965     // Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1])
966 
967     KeyValue [] srcKVs = new KeyValue [] {
968         new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
969         new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
970         new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]),
971         new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1])
972     };
973 
974     for(KeyValue kv : srcKVs) {
975       this.region.put(new Put(kv.getRow()).add(kv));
976     }
977 
978     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false
979     // Expect 3 rows (0, 2, 3)
980     SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILIES[0],
981         QUALIFIERS_ONE[0], CompareOp.EQUAL, VALUES[0]);
982     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
983     s.addFamily(FAMILIES[0]);
984     s.setFilter(scvf);
985     kvs = new KeyValue [] { srcKVs[0], srcKVs[2], srcKVs[3] };
986     verifyScanFull(s, kvs);
987 
988     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true
989     // Expect 1 row (0)
990     scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
991         CompareOp.EQUAL, VALUES[0]);
992     scvf.setFilterIfMissing(true);
993     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
994     s.addFamily(FAMILIES[0]);
995     s.setFilter(scvf);
996     kvs = new KeyValue [] { srcKVs[0] };
997     verifyScanFull(s, kvs);
998 
999     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1000     // Expect 1 row (3)
1001     scvf = new SingleColumnValueFilter(FAMILIES[0],
1002         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1003     scvf.setFilterIfMissing(true);
1004     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1005     s.addFamily(FAMILIES[0]);
1006     s.setFilter(scvf);
1007     kvs = new KeyValue [] { srcKVs[3] };
1008     verifyScanFull(s, kvs);
1009 
1010     // Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0]
1011     KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]);
1012     this.region.put(new Put(kvA.getRow()).add(kvA));
1013 
1014     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1015     // Expect 1 row (3)
1016     scvf = new SingleColumnValueFilter(FAMILIES[0],
1017         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1018     scvf.setFilterIfMissing(true);
1019     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1020     s.addFamily(FAMILIES[0]);
1021     s.setFilter(scvf);
1022     kvs = new KeyValue [] { srcKVs[3] };
1023     verifyScanFull(s, kvs);
1024 
1025   }
1026 
1027   private void verifyScan(Scan s, long expectedRows, long expectedKeys)
1028   throws IOException {
1029     InternalScanner scanner = this.region.getScanner(s);
1030     List<KeyValue> results = new ArrayList<KeyValue>();
1031     int i = 0;
1032     for (boolean done = true; done; i++) {
1033       done = scanner.next(results);
1034       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1035           KeyValue.COMPARATOR);
1036       LOG.info("counter=" + i + ", " + results);
1037       if (results.isEmpty()) break;
1038       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1039           " total but already scanned " + (i+1), expectedRows > i);
1040       assertEquals("Expected " + expectedKeys + " keys per row but " +
1041           "returned " + results.size(), expectedKeys, results.size());
1042       results.clear();
1043     }
1044     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1045         " rows", expectedRows, i);
1046   }
1047 
1048 
1049 
1050   private void verifyScanNoEarlyOut(Scan s, long expectedRows,
1051       long expectedKeys)
1052   throws IOException {
1053     InternalScanner scanner = this.region.getScanner(s);
1054     List<KeyValue> results = new ArrayList<KeyValue>();
1055     int i = 0;
1056     for (boolean done = true; done; i++) {
1057       done = scanner.next(results);
1058       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1059           KeyValue.COMPARATOR);
1060       LOG.info("counter=" + i + ", " + results);
1061       if(results.isEmpty()) break;
1062       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1063           " total but already scanned " + (i+1), expectedRows > i);
1064       assertEquals("Expected " + expectedKeys + " keys per row but " +
1065           "returned " + results.size(), expectedKeys, results.size());
1066       results.clear();
1067     }
1068     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1069         " rows", expectedRows, i);
1070   }
1071 
1072   private void verifyScanFull(Scan s, KeyValue [] kvs)
1073   throws IOException {
1074     InternalScanner scanner = this.region.getScanner(s);
1075     List<KeyValue> results = new ArrayList<KeyValue>();
1076     int row = 0;
1077     int idx = 0;
1078     for (boolean done = true; done; row++) {
1079       done = scanner.next(results);
1080       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1081           KeyValue.COMPARATOR);
1082       if(results.isEmpty()) break;
1083       assertTrue("Scanned too many keys! Only expected " + kvs.length +
1084           " total but already scanned " + (results.size() + idx) +
1085           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
1086           kvs.length >= idx + results.size());
1087       for(KeyValue kv : results) {
1088         LOG.info("row=" + row + ", result=" + kv.toString() +
1089             ", match=" + kvs[idx].toString());
1090         assertTrue("Row mismatch",
1091             Bytes.equals(kv.getRow(), kvs[idx].getRow()));
1092         assertTrue("Family mismatch",
1093             Bytes.equals(kv.getFamily(), kvs[idx].getFamily()));
1094         assertTrue("Qualifier mismatch",
1095             Bytes.equals(kv.getQualifier(), kvs[idx].getQualifier()));
1096         assertTrue("Value mismatch",
1097             Bytes.equals(kv.getValue(), kvs[idx].getValue()));
1098         idx++;
1099       }
1100       results.clear();
1101     }
1102     LOG.info("Looked at " + row + " rows with " + idx + " keys");
1103     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
1104         kvs.length, idx);
1105   }
1106 
1107 
1108   public void testColumnPaginationFilter() throws Exception {
1109 
1110      // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
1111       KeyValue [] expectedKVs = {
1112         // testRowOne-0
1113         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1114         // testRowOne-2
1115         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1116         // testRowOne-3
1117         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1118         // testRowTwo-0
1119         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1120         // testRowTwo-2
1121         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1122         // testRowTwo-3
1123         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
1124       };
1125 
1126 
1127       // Set of KVs (page: 3; pageSize: 1)  - the third set of 1 column per row
1128       KeyValue [] expectedKVs2 = {
1129         // testRowOne-0
1130         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1131         // testRowOne-2
1132         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1133         // testRowOne-3
1134         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1135         // testRowTwo-0
1136         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1137         // testRowTwo-2
1138         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1139         // testRowTwo-3
1140         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1141       };
1142 
1143       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1144       KeyValue [] expectedKVs3 = {
1145         // testRowOne-0
1146         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1147         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1148         // testRowOne-2
1149         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1150         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1151         // testRowOne-3
1152         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1153         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1154         // testRowTwo-0
1155         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1156         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1157         // testRowTwo-2
1158         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1159         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1160         // testRowTwo-3
1161         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1162         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1163       };
1164 
1165 
1166       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1167       KeyValue [] expectedKVs4 = {
1168 
1169       };
1170 
1171       long expectedRows = this.numRows;
1172       long expectedKeys = 1;
1173       Scan s = new Scan();
1174 
1175 
1176       // Page 1; 1 Column per page  (Limit 1, Offset 0)
1177       s.setFilter(new ColumnPaginationFilter(1,0));
1178       verifyScan(s, expectedRows, expectedKeys);
1179       this.verifyScanFull(s, expectedKVs);
1180 
1181       // Page 3; 1 Result per page  (Limit 1, Offset 2)
1182       s.setFilter(new ColumnPaginationFilter(1,2));
1183       verifyScan(s, expectedRows, expectedKeys);
1184       this.verifyScanFull(s, expectedKVs2);
1185 
1186       // Page 2; 2 Results per page (Limit 2, Offset 2)
1187       s.setFilter(new ColumnPaginationFilter(2,2));
1188       expectedKeys = 2;
1189       verifyScan(s, expectedRows, expectedKeys);
1190       this.verifyScanFull(s, expectedKVs3);
1191 
1192       // Page 8; 20 Results per page (no results) (Limit 20, Offset 140)
1193       s.setFilter(new ColumnPaginationFilter(20,140));
1194       expectedKeys = 0;
1195       expectedRows = 0;
1196       verifyScan(s, expectedRows, 0);
1197       this.verifyScanFull(s, expectedKVs4);
1198     }
1199 }