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