View Javadoc

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