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