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 testFamilyFilter() throws IOException {
558 
559       // Match family, only half of columns returned.
560       long expectedRows = this.numRows;
561       long expectedKeys = this.colsPerRow / 2;
562       Filter f = new FamilyFilter(CompareOp.EQUAL,
563           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
564       Scan s = new Scan();
565       s.setFilter(f);
566       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
567 
568       // Match keys less than given family, should return nothing
569       expectedRows = 0;
570       expectedKeys = 0;
571       f = new FamilyFilter(CompareOp.LESS,
572           new BinaryComparator(Bytes.toBytes("testFamily")));
573       s = new Scan();
574       s.setFilter(f);
575       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
576 
577       // Match keys less than or equal, should return half of columns
578       expectedRows = this.numRows;
579       expectedKeys = this.colsPerRow / 2;
580       f = new FamilyFilter(CompareOp.LESS_OR_EQUAL,
581           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
582       s = new Scan();
583       s.setFilter(f);
584       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
585 
586       // Match keys from second family
587       // look only in second group of rows
588       expectedRows = this.numRows / 2;
589       expectedKeys = this.colsPerRow / 2;
590       f = new FamilyFilter(CompareOp.NOT_EQUAL,
591           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
592       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
593       s.setFilter(f);
594       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
595 
596       // Match all columns
597       // look only in second group of rows
598       expectedRows = this.numRows / 2;
599       expectedKeys = this.colsPerRow;
600       f = new FamilyFilter(CompareOp.GREATER_OR_EQUAL,
601           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
602       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
603       s.setFilter(f);
604       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
605 
606       // Match all columns in second family
607       // look only in second group of rows        
608       expectedRows = this.numRows / 2;
609       expectedKeys = this.colsPerRow / 2;
610       f = new FamilyFilter(CompareOp.GREATER,
611           new BinaryComparator(Bytes.toBytes("testFamilyOne")));
612       s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
613       s.setFilter(f);
614       verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
615 
616       // Match keys not equal to given family
617       // Look across rows and fully validate the keys and ordering
618       f = new FamilyFilter(CompareOp.NOT_EQUAL,
619           new BinaryComparator(FAMILIES[1]));
620       s = new Scan();
621       s.setFilter(f);
622 
623       KeyValue [] kvs = {
624           // testRowOne-0
625           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
626           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
627           new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
628           // testRowOne-2
629           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
630           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
631           new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
632           // testRowOne-3
633           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
634           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
635           new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
636           // testRowTwo-0
637           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
638           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
639           new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
640           // testRowTwo-2
641           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
642           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
643           new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
644           // testRowTwo-3
645           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
646           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
647           new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
648       };
649       verifyScanFull(s, kvs);
650 
651 
652       // Test across rows and groups with a regex
653       // Filter out "test*-2"
654       // Expect 4 keys per row across both groups
655       f = new FamilyFilter(CompareOp.NOT_EQUAL,
656           new RegexStringComparator("test.*One"));
657       s = new Scan();
658       s.setFilter(f);
659 
660       kvs = new KeyValue [] {
661           // testRowOne-0
662           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
663           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
664           new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
665           // testRowOne-2
666           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
667           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
668           new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
669           // testRowOne-3
670           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
671           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
672           new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
673           // testRowTwo-0
674           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
675           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
676           new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
677           // testRowTwo-2
678           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
679           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
680           new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
681           // testRowTwo-3
682           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
683           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
684           new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
685       };
686       verifyScanFull(s, kvs);
687 
688     }
689 
690 
691   public void testRowFilter() throws IOException {
692 
693     // Match a single row, all keys
694     long expectedRows = 1;
695     long expectedKeys = this.colsPerRow;
696     Filter f = new RowFilter(CompareOp.EQUAL,
697         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
698     Scan s = new Scan();
699     s.setFilter(f);
700     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
701 
702     // Match a two rows, one from each group, using regex
703     expectedRows = 2;
704     expectedKeys = this.colsPerRow;
705     f = new RowFilter(CompareOp.EQUAL,
706         new RegexStringComparator("testRow.+-2"));
707     s = new Scan();
708     s.setFilter(f);
709     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
710 
711     // Match rows less than
712     // Expect all keys in one row
713     expectedRows = 1;
714     expectedKeys = this.colsPerRow;
715     f = new RowFilter(CompareOp.LESS,
716         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
717     s = new Scan();
718     s.setFilter(f);
719     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
720 
721     // Match rows less than or equal
722     // Expect all keys in two rows
723     expectedRows = 2;
724     expectedKeys = this.colsPerRow;
725     f = new RowFilter(CompareOp.LESS_OR_EQUAL,
726         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
727     s = new Scan();
728     s.setFilter(f);
729     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
730 
731     // Match rows not equal
732     // Expect all keys in all but one row
733     expectedRows = this.numRows - 1;
734     expectedKeys = this.colsPerRow;
735     f = new RowFilter(CompareOp.NOT_EQUAL,
736         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
737     s = new Scan();
738     s.setFilter(f);
739     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
740 
741     // Match keys greater or equal
742     // Expect all keys in all but one row
743     expectedRows = this.numRows - 1;
744     expectedKeys = this.colsPerRow;
745     f = new RowFilter(CompareOp.GREATER_OR_EQUAL,
746         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
747     s = new Scan();
748     s.setFilter(f);
749     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
750 
751     // Match keys greater
752     // Expect all keys in all but two rows
753     expectedRows = this.numRows - 2;
754     expectedKeys = this.colsPerRow;
755     f = new RowFilter(CompareOp.GREATER,
756         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
757     s = new Scan();
758     s.setFilter(f);
759     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
760 
761     // Match rows not equal to testRowTwo-2
762     // Look across rows and fully validate the keys and ordering
763     // Should see all keys in all rows but testRowTwo-2
764     f = new RowFilter(CompareOp.NOT_EQUAL,
765         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
766     s = new Scan();
767     s.setFilter(f);
768 
769     KeyValue [] kvs = {
770         // testRowOne-0
771         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
772         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
773         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
774         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
775         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
776         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
777         // testRowOne-3
778         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
779         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
780         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
781         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
782         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
783         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
784         // testRowTwo-0
785         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
786         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
787         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
788         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
789         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
790         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
791         // testRowTwo-2
792         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
793         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
794         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
795         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
796         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
797         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
798         // testRowTwo-3
799         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
800         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
801         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
802         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
803         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
804         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
805     };
806     verifyScanFull(s, kvs);
807 
808 
809     // Test across rows and groups with a regex
810     // Filter out everything that doesn't match "*-2"
811     // Expect all keys in two rows
812     f = new RowFilter(CompareOp.EQUAL,
813         new RegexStringComparator(".+-2"));
814     s = new Scan();
815     s.setFilter(f);
816 
817     kvs = new KeyValue [] {
818         // testRowOne-2
819         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
820         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
821         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
822         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
823         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
824         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
825         // testRowTwo-2
826         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
827         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
828         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
829         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
830         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
831         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
832     };
833     verifyScanFull(s, kvs);
834 
835   }
836 
837   public void testValueFilter() throws IOException {
838 
839     // Match group one rows
840     long expectedRows = this.numRows / 2;
841     long expectedKeys = this.colsPerRow;
842     Filter f = new ValueFilter(CompareOp.EQUAL,
843         new BinaryComparator(Bytes.toBytes("testValueOne")));
844     Scan s = new Scan();
845     s.setFilter(f);
846     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
847 
848     // Match group two rows
849     expectedRows = this.numRows / 2;
850     expectedKeys = this.colsPerRow;
851     f = new ValueFilter(CompareOp.EQUAL,
852         new BinaryComparator(Bytes.toBytes("testValueTwo")));
853     s = new Scan();
854     s.setFilter(f);
855     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
856 
857     // Match all values using regex
858     expectedRows = this.numRows;
859     expectedKeys = this.colsPerRow;
860     f = new ValueFilter(CompareOp.EQUAL,
861         new RegexStringComparator("testValue((One)|(Two))"));
862     s = new Scan();
863     s.setFilter(f);
864     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
865 
866     // Match values less than
867     // Expect group one rows
868     expectedRows = this.numRows / 2;
869     expectedKeys = this.colsPerRow;
870     f = new ValueFilter(CompareOp.LESS,
871         new BinaryComparator(Bytes.toBytes("testValueTwo")));
872     s = new Scan();
873     s.setFilter(f);
874     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
875 
876     // Match values less than or equal
877     // Expect all rows
878     expectedRows = this.numRows;
879     expectedKeys = this.colsPerRow;
880     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
881         new BinaryComparator(Bytes.toBytes("testValueTwo")));
882     s = new Scan();
883     s.setFilter(f);
884     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
885 
886     // Match values less than or equal
887     // Expect group one rows
888     expectedRows = this.numRows / 2;
889     expectedKeys = this.colsPerRow;
890     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
891         new BinaryComparator(Bytes.toBytes("testValueOne")));
892     s = new Scan();
893     s.setFilter(f);
894     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
895 
896     // Match values not equal
897     // Expect half the rows
898     expectedRows = this.numRows / 2;
899     expectedKeys = this.colsPerRow;
900     f = new ValueFilter(CompareOp.NOT_EQUAL,
901         new BinaryComparator(Bytes.toBytes("testValueOne")));
902     s = new Scan();
903     s.setFilter(f);
904     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
905 
906     // Match values greater or equal
907     // Expect all rows
908     expectedRows = this.numRows;
909     expectedKeys = this.colsPerRow;
910     f = new ValueFilter(CompareOp.GREATER_OR_EQUAL,
911         new BinaryComparator(Bytes.toBytes("testValueOne")));
912     s = new Scan();
913     s.setFilter(f);
914     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
915 
916     // Match values greater
917     // Expect half rows
918     expectedRows = this.numRows / 2;
919     expectedKeys = this.colsPerRow;
920     f = new ValueFilter(CompareOp.GREATER,
921         new BinaryComparator(Bytes.toBytes("testValueOne")));
922     s = new Scan();
923     s.setFilter(f);
924     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
925 
926     // Match values not equal to testValueOne
927     // Look across rows and fully validate the keys and ordering
928     // Should see all keys in all group two rows
929     f = new ValueFilter(CompareOp.NOT_EQUAL,
930         new BinaryComparator(Bytes.toBytes("testValueOne")));
931     s = new Scan();
932     s.setFilter(f);
933 
934     KeyValue [] kvs = {
935         // testRowTwo-0
936         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
937         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
938         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
939         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
940         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
941         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
942         // testRowTwo-2
943         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
944         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
945         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
946         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
947         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
948         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
949         // testRowTwo-3
950         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
951         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
952         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
953         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
954         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
955         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
956     };
957     verifyScanFull(s, kvs);
958   }
959 
960   public void testSkipFilter() throws IOException {
961 
962     // Test for qualifier regex: "testQualifierOne-2"
963     // Should only get rows from second group, and all keys
964     Filter f = new SkipFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
965         new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))));
966     Scan s = new Scan();
967     s.setFilter(f);
968 
969     KeyValue [] kvs = {
970         // testRowTwo-0
971         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
972         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
973         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
974         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
975         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
976         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
977         // testRowTwo-2
978         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
979         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
980         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
981         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
982         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
983         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
984         // testRowTwo-3
985         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
986         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
987         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
988         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
989         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
990         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
991     };
992     verifyScanFull(s, kvs);
993   }
994 
995   // TODO: This is important... need many more tests for ordering, etc
996   // There are limited tests elsewhere but we need HRegion level ones here
997   public void testFilterList() throws IOException {
998 
999     // Test getting a single row, single key using Row, Qualifier, and Value
1000     // regular expression and substring filters
1001     // Use must pass all
1002     List<Filter> filters = new ArrayList<Filter>();
1003     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1004     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1005     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
1006     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1007     Scan s = new Scan();
1008     s.addFamily(FAMILIES[0]);
1009     s.setFilter(f);
1010     KeyValue [] kvs = {
1011         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0])
1012     };
1013     verifyScanFull(s, kvs);
1014 
1015     // Test getting everything with a MUST_PASS_ONE filter including row, qf, val
1016     // regular expression and substring filters
1017     filters.clear();
1018     filters.add(new RowFilter(CompareOp.EQUAL, new RegexStringComparator(".+Two.+")));
1019     filters.add(new QualifierFilter(CompareOp.EQUAL, new RegexStringComparator(".+-2")));
1020     filters.add(new ValueFilter(CompareOp.EQUAL, new SubstringComparator("One")));
1021     f = new FilterList(Operator.MUST_PASS_ONE, filters);
1022     s = new Scan();
1023     s.setFilter(f);
1024     verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow);
1025 
1026 
1027   }
1028 
1029   public void testFirstKeyOnlyFilter() throws IOException {
1030     Scan s = new Scan();
1031     s.setFilter(new FirstKeyOnlyFilter());
1032     // Expected KVs, the first KV from each of the remaining 6 rows
1033     KeyValue [] kvs = {
1034         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1035         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1036         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1037         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1038         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1039         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
1040     };
1041     verifyScanFull(s, kvs);
1042   }
1043 
1044   public void testFilterListWithSingleColumnValueFilter() throws IOException {
1045     // Test for HBASE-3191
1046 
1047     // Scan using SingleColumnValueFilter
1048     SingleColumnValueFilter f1 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1049           CompareOp.EQUAL, VALUES[0]);
1050     f1.setFilterIfMissing( true );
1051     Scan s1 = new Scan();
1052     s1.addFamily(FAMILIES[0]);
1053     s1.setFilter(f1);
1054     KeyValue [] kvs1 = {
1055         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1056         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1057         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1058         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1059         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1060         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1061         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1062         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1063         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1064     };
1065     verifyScanNoEarlyOut(s1, 3, 3);
1066     verifyScanFull(s1, kvs1);
1067 
1068     // Scan using another SingleColumnValueFilter, expect disjoint result
1069     SingleColumnValueFilter f2 = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_TWO[0],
1070         CompareOp.EQUAL, VALUES[1]);
1071     f2.setFilterIfMissing( true );
1072     Scan s2 = new Scan();
1073     s2.addFamily(FAMILIES[0]);
1074     s2.setFilter(f2);
1075     KeyValue [] kvs2 = {
1076         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1077         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1078         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1079         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1080         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1081         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1082         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1083         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1084         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1085     };
1086     verifyScanNoEarlyOut(s2, 3, 3);
1087     verifyScanFull(s2, kvs2);
1088 
1089     // Scan, ORing the two previous filters, expect unified result
1090     FilterList f = new FilterList(Operator.MUST_PASS_ONE);
1091     f.addFilter(f1);
1092     f.addFilter(f2);
1093     Scan s = new Scan();
1094     s.addFamily(FAMILIES[0]);
1095     s.setFilter(f);
1096     KeyValue [] kvs = {
1097         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1098         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1099         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1100         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1101         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1102         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1103         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1104         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1105         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1106         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1107         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1108         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1109         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1110         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1111         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1112         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1113         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1114         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1115     };
1116     verifyScanNoEarlyOut(s, 6, 3);
1117     verifyScanFull(s, kvs);
1118   }
1119 
1120   public void testSingleColumnValueFilter() throws IOException {
1121 
1122     // From HBASE-1821
1123     // Desired action is to combine two SCVF in a FilterList
1124     // Want to return only rows that match both conditions
1125 
1126     // Need to change one of the group one columns to use group two value
1127     Put p = new Put(ROWS_ONE[2]);
1128     p.add(FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]);
1129     this.region.put(p);
1130 
1131     // Now let's grab rows that have Q_ONE[0](VALUES[0]) and Q_ONE[2](VALUES[1])
1132     // Since group two rows don't have these qualifiers, they will pass
1133     // so limiting scan to group one
1134     List<Filter> filters = new ArrayList<Filter>();
1135     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1136         CompareOp.EQUAL, VALUES[0]));
1137     filters.add(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
1138         CompareOp.EQUAL, VALUES[1]));
1139     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
1140     Scan s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
1141     s.addFamily(FAMILIES[0]);
1142     s.setFilter(f);
1143     // Expect only one row, all qualifiers
1144     KeyValue [] kvs = {
1145         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1146         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[1]),
1147         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0])
1148     };
1149     verifyScanNoEarlyOut(s, 1, 3);
1150     verifyScanFull(s, kvs);
1151 
1152     // In order to get expected behavior without limiting to group one
1153     // need to wrap SCVFs in SkipFilters
1154     filters = new ArrayList<Filter>();
1155     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1156         CompareOp.EQUAL, VALUES[0])));
1157     filters.add(new SkipFilter(new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[2],
1158         CompareOp.EQUAL, VALUES[1])));
1159     f = new FilterList(Operator.MUST_PASS_ALL, filters);
1160     s = new Scan(ROWS_ONE[0], ROWS_TWO[0]);
1161     s.addFamily(FAMILIES[0]);
1162     s.setFilter(f);
1163     // Expect same KVs
1164     verifyScanNoEarlyOut(s, 1, 3);
1165     verifyScanFull(s, kvs);
1166 
1167     // More tests from HBASE-1821 for Clint and filterIfMissing flag
1168 
1169     byte [][] ROWS_THREE = {
1170         Bytes.toBytes("rowThree-0"), Bytes.toBytes("rowThree-1"),
1171         Bytes.toBytes("rowThree-2"), Bytes.toBytes("rowThree-3")
1172     };
1173 
1174     // Give row 0 and 2 QUALIFIERS_ONE[0] (VALUE[0] VALUE[1])
1175     // Give row 1 and 3 QUALIFIERS_ONE[1] (VALUE[0] VALUE[1])
1176 
1177     KeyValue [] srcKVs = new KeyValue [] {
1178         new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1179         new KeyValue(ROWS_THREE[1], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[1]),
1180         new KeyValue(ROWS_THREE[2], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]),
1181         new KeyValue(ROWS_THREE[3], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[1])
1182     };
1183 
1184     for(KeyValue kv : srcKVs) {
1185       this.region.put(new Put(kv.getRow()).add(kv));
1186     }
1187 
1188     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = false
1189     // Expect 3 rows (0, 2, 3)
1190     SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILIES[0],
1191         QUALIFIERS_ONE[0], CompareOp.EQUAL, VALUES[0]);
1192     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1193     s.addFamily(FAMILIES[0]);
1194     s.setFilter(scvf);
1195     kvs = new KeyValue [] { srcKVs[0], srcKVs[2], srcKVs[3] };
1196     verifyScanFull(s, kvs);
1197 
1198     // Match VALUES[0] against QUALIFIERS_ONE[0] with filterIfMissing = true
1199     // Expect 1 row (0)
1200     scvf = new SingleColumnValueFilter(FAMILIES[0], QUALIFIERS_ONE[0],
1201         CompareOp.EQUAL, VALUES[0]);
1202     scvf.setFilterIfMissing(true);
1203     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1204     s.addFamily(FAMILIES[0]);
1205     s.setFilter(scvf);
1206     kvs = new KeyValue [] { srcKVs[0] };
1207     verifyScanFull(s, kvs);
1208 
1209     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1210     // Expect 1 row (3)
1211     scvf = new SingleColumnValueFilter(FAMILIES[0],
1212         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1213     scvf.setFilterIfMissing(true);
1214     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1215     s.addFamily(FAMILIES[0]);
1216     s.setFilter(scvf);
1217     kvs = new KeyValue [] { srcKVs[3] };
1218     verifyScanFull(s, kvs);
1219 
1220     // Add QUALIFIERS_ONE[1] to ROWS_THREE[0] with VALUES[0]
1221     KeyValue kvA = new KeyValue(ROWS_THREE[0], FAMILIES[0], QUALIFIERS_ONE[1], VALUES[0]);
1222     this.region.put(new Put(kvA.getRow()).add(kvA));
1223 
1224     // Match VALUES[1] against QUALIFIERS_ONE[1] with filterIfMissing = true
1225     // Expect 1 row (3)
1226     scvf = new SingleColumnValueFilter(FAMILIES[0],
1227         QUALIFIERS_ONE[1], CompareOp.EQUAL, VALUES[1]);
1228     scvf.setFilterIfMissing(true);
1229     s = new Scan(ROWS_THREE[0], Bytes.toBytes("rowThree-4"));
1230     s.addFamily(FAMILIES[0]);
1231     s.setFilter(scvf);
1232     kvs = new KeyValue [] { srcKVs[3] };
1233     verifyScanFull(s, kvs);
1234 
1235   }
1236 
1237   private void verifyScan(Scan s, long expectedRows, long expectedKeys)
1238   throws IOException {
1239     InternalScanner scanner = this.region.getScanner(s);
1240     List<KeyValue> results = new ArrayList<KeyValue>();
1241     int i = 0;
1242     for (boolean done = true; done; i++) {
1243       done = scanner.next(results);
1244       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1245           KeyValue.COMPARATOR);
1246       LOG.info("counter=" + i + ", " + results);
1247       if (results.isEmpty()) break;
1248       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1249           " total but already scanned " + (i+1), expectedRows > i);
1250       assertEquals("Expected " + expectedKeys + " keys per row but " +
1251           "returned " + results.size(), expectedKeys, results.size());
1252       results.clear();
1253     }
1254     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1255         " rows", expectedRows, i);
1256   }
1257 
1258   private void verifyScanNoEarlyOut(Scan s, long expectedRows,
1259       long expectedKeys)
1260   throws IOException {
1261     InternalScanner scanner = this.region.getScanner(s);
1262     List<KeyValue> results = new ArrayList<KeyValue>();
1263     int i = 0;
1264     for (boolean done = true; done; i++) {
1265       done = scanner.next(results);
1266       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1267           KeyValue.COMPARATOR);
1268       LOG.info("counter=" + i + ", " + results);
1269       if(results.isEmpty()) break;
1270       assertTrue("Scanned too many rows! Only expected " + expectedRows +
1271           " total but already scanned " + (i+1), expectedRows > i);
1272       assertEquals("Expected " + expectedKeys + " keys per row but " +
1273           "returned " + results.size(), expectedKeys, results.size());
1274       results.clear();
1275     }
1276     assertEquals("Expected " + expectedRows + " rows but scanned " + i +
1277         " rows", expectedRows, i);
1278   }
1279 
1280   private void verifyScanFull(Scan s, KeyValue [] kvs)
1281   throws IOException {
1282     InternalScanner scanner = this.region.getScanner(s);
1283     List<KeyValue> results = new ArrayList<KeyValue>();
1284     int row = 0;
1285     int idx = 0;
1286     for (boolean done = true; done; row++) {
1287       done = scanner.next(results);
1288       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1289           KeyValue.COMPARATOR);
1290       if(results.isEmpty()) break;
1291       assertTrue("Scanned too many keys! Only expected " + kvs.length +
1292           " total but already scanned " + (results.size() + idx) +
1293           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
1294           kvs.length >= idx + results.size());
1295       for(KeyValue kv : results) {
1296         LOG.info("row=" + row + ", result=" + kv.toString() +
1297             ", match=" + kvs[idx].toString());
1298         assertTrue("Row mismatch",
1299             Bytes.equals(kv.getRow(), kvs[idx].getRow()));
1300         assertTrue("Family mismatch",
1301             Bytes.equals(kv.getFamily(), kvs[idx].getFamily()));
1302         assertTrue("Qualifier mismatch",
1303             Bytes.equals(kv.getQualifier(), kvs[idx].getQualifier()));
1304         assertTrue("Value mismatch",
1305             Bytes.equals(kv.getValue(), kvs[idx].getValue()));
1306         idx++;
1307       }
1308       results.clear();
1309     }
1310     LOG.info("Looked at " + row + " rows with " + idx + " keys");
1311     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
1312         kvs.length, idx);
1313   }
1314 
1315   private void verifyScanFullNoValues(Scan s, KeyValue [] kvs, boolean useLen)
1316   throws IOException {
1317     InternalScanner scanner = this.region.getScanner(s);
1318     List<KeyValue> results = new ArrayList<KeyValue>();
1319     int row = 0;
1320     int idx = 0;
1321     for (boolean more = true; more; row++) {
1322       more = scanner.next(results);
1323       Arrays.sort(results.toArray(new KeyValue[results.size()]),
1324           KeyValue.COMPARATOR);
1325       if(results.isEmpty()) break;
1326       assertTrue("Scanned too many keys! Only expected " + kvs.length +
1327           " total but already scanned " + (results.size() + idx) +
1328           (results.isEmpty() ? "" : "(" + results.get(0).toString() + ")"),
1329           kvs.length >= idx + results.size());
1330       for(KeyValue kv : results) {
1331         LOG.info("row=" + row + ", result=" + kv.toString() +
1332             ", match=" + kvs[idx].toString());
1333         assertTrue("Row mismatch",
1334             Bytes.equals(kv.getRow(), kvs[idx].getRow()));
1335         assertTrue("Family mismatch",
1336             Bytes.equals(kv.getFamily(), kvs[idx].getFamily()));
1337         assertTrue("Qualifier mismatch",
1338             Bytes.equals(kv.getQualifier(), kvs[idx].getQualifier()));
1339         assertFalse("Should not have returned whole value",
1340             Bytes.equals(kv.getValue(), kvs[idx].getValue()));
1341         if (useLen) {
1342           assertEquals("Value in result is not SIZEOF_INT", 
1343                      kv.getValue().length, Bytes.SIZEOF_INT);
1344           LOG.info("idx = "  + idx + ", len=" + kvs[idx].getValueLength()
1345               + ", actual=" +  Bytes.toInt(kv.getValue()));
1346           assertEquals("Scan value should be the length of the actual value. ",
1347                      kvs[idx].getValueLength(), Bytes.toInt(kv.getValue()) );
1348           LOG.info("good");
1349         } else {
1350           assertEquals("Value in result is not empty", 
1351                      kv.getValue().length, 0);
1352         }
1353         idx++;
1354       }
1355       results.clear();
1356     }
1357     LOG.info("Looked at " + row + " rows with " + idx + " keys");
1358     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
1359         kvs.length, idx);
1360   }
1361 
1362 
1363   public void testColumnPaginationFilter() throws Exception {
1364 
1365      // Set of KVs (page: 1; pageSize: 1) - the first set of 1 column per row
1366       KeyValue [] expectedKVs = {
1367         // testRowOne-0
1368         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1369         // testRowOne-2
1370         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1371         // testRowOne-3
1372         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1373         // testRowTwo-0
1374         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1375         // testRowTwo-2
1376         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1377         // testRowTwo-3
1378         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
1379       };
1380 
1381 
1382       // Set of KVs (page: 3; pageSize: 1)  - the third set of 1 column per row
1383       KeyValue [] expectedKVs2 = {
1384         // testRowOne-0
1385         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1386         // testRowOne-2
1387         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1388         // testRowOne-3
1389         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1390         // testRowTwo-0
1391         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1392         // testRowTwo-2
1393         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1394         // testRowTwo-3
1395         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1396       };
1397 
1398       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1399       KeyValue [] expectedKVs3 = {
1400         // testRowOne-0
1401         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1402         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1403         // testRowOne-2
1404         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1405         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1406         // testRowOne-3
1407         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1408         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1409         // testRowTwo-0
1410         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1411         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1412         // testRowTwo-2
1413         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1414         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1415         // testRowTwo-3
1416         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1417         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1418       };
1419 
1420 
1421       // Set of KVs (page: 2; pageSize 2)  - the 2nd set of 2 columns per row
1422       KeyValue [] expectedKVs4 = {
1423 
1424       };
1425 
1426       long expectedRows = this.numRows;
1427       long expectedKeys = 1;
1428       Scan s = new Scan();
1429 
1430 
1431       // Page 1; 1 Column per page  (Limit 1, Offset 0)
1432       s.setFilter(new ColumnPaginationFilter(1,0));
1433       verifyScan(s, expectedRows, expectedKeys);
1434       this.verifyScanFull(s, expectedKVs);
1435 
1436       // Page 3; 1 Result per page  (Limit 1, Offset 2)
1437       s.setFilter(new ColumnPaginationFilter(1,2));
1438       verifyScan(s, expectedRows, expectedKeys);
1439       this.verifyScanFull(s, expectedKVs2);
1440 
1441       // Page 2; 2 Results per page (Limit 2, Offset 2)
1442       s.setFilter(new ColumnPaginationFilter(2,2));
1443       expectedKeys = 2;
1444       verifyScan(s, expectedRows, expectedKeys);
1445       this.verifyScanFull(s, expectedKVs3);
1446 
1447       // Page 8; 20 Results per page (no results) (Limit 20, Offset 140)
1448       s.setFilter(new ColumnPaginationFilter(20,140));
1449       expectedKeys = 0;
1450       expectedRows = 0;
1451       verifyScan(s, expectedRows, 0);
1452       this.verifyScanFull(s, expectedKVs4);
1453     }
1454 
1455   public void testKeyOnlyFilter() throws Exception {
1456 
1457     // KVs in first 6 rows
1458     KeyValue [] expectedKVs = {
1459       // testRowOne-0
1460       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1461       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1462       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1463       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1464       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1465       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1466       // testRowOne-2
1467       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1468       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1469       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1470       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1471       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1472       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1473       // testRowOne-3
1474       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
1475       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
1476       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
1477       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
1478       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
1479       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
1480       // testRowTwo-0
1481       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1482       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1483       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1484       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1485       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1486       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1487       // testRowTwo-2
1488       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1489       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1490       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1491       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1492       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1493       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
1494       // testRowTwo-3
1495       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
1496       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
1497       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
1498       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
1499       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
1500       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
1501     };
1502 
1503     // Grab all 6 rows
1504     long expectedRows = 6;
1505     long expectedKeys = this.colsPerRow;
1506     for (boolean useLen : new boolean[]{false,true}) {
1507       Scan s = new Scan();
1508       s.setFilter(new KeyOnlyFilter(useLen));
1509       verifyScan(s, expectedRows, expectedKeys);
1510       verifyScanFullNoValues(s, expectedKVs, useLen);
1511     }
1512   }
1513 }