1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  
21  package org.apache.hadoop.hbase.rest;
22  
23  import java.io.ByteArrayInputStream;
24  import java.io.StringWriter;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Iterator;
28  import java.util.List;
29  
30  import javax.xml.bind.JAXBContext;
31  import javax.xml.bind.Marshaller;
32  import javax.xml.bind.Unmarshaller;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.hbase.HColumnDescriptor;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HTableDescriptor;
39  import org.apache.hadoop.hbase.KeyValue;
40  import org.apache.hadoop.hbase.client.Delete;
41  import org.apache.hadoop.hbase.client.HBaseAdmin;
42  import org.apache.hadoop.hbase.client.HTable;
43  import org.apache.hadoop.hbase.client.Put;
44  import org.apache.hadoop.hbase.client.Scan;
45  import org.apache.hadoop.hbase.filter.BinaryComparator;
46  import org.apache.hadoop.hbase.filter.Filter;
47  import org.apache.hadoop.hbase.filter.FilterList;
48  import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
49  import org.apache.hadoop.hbase.filter.InclusiveStopFilter;
50  import org.apache.hadoop.hbase.filter.PageFilter;
51  import org.apache.hadoop.hbase.filter.PrefixFilter;
52  import org.apache.hadoop.hbase.filter.QualifierFilter;
53  import org.apache.hadoop.hbase.filter.RegexStringComparator;
54  import org.apache.hadoop.hbase.filter.RowFilter;
55  import org.apache.hadoop.hbase.filter.SkipFilter;
56  import org.apache.hadoop.hbase.filter.SubstringComparator;
57  import org.apache.hadoop.hbase.filter.ValueFilter;
58  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
59  import org.apache.hadoop.hbase.filter.FilterList.Operator;
60  import org.apache.hadoop.hbase.rest.client.Client;
61  import org.apache.hadoop.hbase.rest.client.Cluster;
62  import org.apache.hadoop.hbase.rest.client.Response;
63  import org.apache.hadoop.hbase.rest.model.CellModel;
64  import org.apache.hadoop.hbase.rest.model.CellSetModel;
65  import org.apache.hadoop.hbase.rest.model.RowModel;
66  import org.apache.hadoop.hbase.rest.model.ScannerModel;
67  import org.apache.hadoop.hbase.util.Bytes;
68  
69  public class TestScannersWithFilters extends HBaseRESTClusterTestBase {
70  
71    static final Log LOG = LogFactory.getLog(TestScannersWithFilters.class);
72  
73    static final byte [][] ROWS_ONE = {
74      Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
75      Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3")
76    };
77  
78    static final byte [][] ROWS_TWO = {
79      Bytes.toBytes("testRowTwo-0"), Bytes.toBytes("testRowTwo-1"),
80      Bytes.toBytes("testRowTwo-2"), Bytes.toBytes("testRowTwo-3")
81    };
82  
83    static final byte [][] FAMILIES = {
84      Bytes.toBytes("testFamilyOne"), Bytes.toBytes("testFamilyTwo")
85    };
86  
87    static final byte [][] QUALIFIERS_ONE = {
88      Bytes.toBytes("testQualifierOne-0"), Bytes.toBytes("testQualifierOne-1"),
89      Bytes.toBytes("testQualifierOne-2"), Bytes.toBytes("testQualifierOne-3")
90    };
91  
92    static final byte [][] QUALIFIERS_TWO = {
93      Bytes.toBytes("testQualifierTwo-0"), Bytes.toBytes("testQualifierTwo-1"),
94      Bytes.toBytes("testQualifierTwo-2"), Bytes.toBytes("testQualifierTwo-3")
95    };
96  
97    static final byte [][] VALUES = {
98      Bytes.toBytes("testValueOne"), Bytes.toBytes("testValueTwo")
99    };
100 
101   Client client;
102   JAXBContext context;
103   Marshaller marshaller;
104   Unmarshaller unmarshaller;
105   long numRows = ROWS_ONE.length + ROWS_TWO.length;
106   long colsPerRow = FAMILIES.length * QUALIFIERS_ONE.length;
107 
108   @Override
109   protected void setUp() throws Exception {
110     super.setUp();
111     context = JAXBContext.newInstance(
112         CellModel.class,
113         CellSetModel.class,
114         RowModel.class,
115         ScannerModel.class);
116     marshaller = context.createMarshaller();
117     unmarshaller = context.createUnmarshaller();
118     client = new Client(new Cluster().add("localhost", testServletPort));
119     HBaseAdmin admin = new HBaseAdmin(conf);
120     if (!admin.tableExists(getName())) {
121       HTableDescriptor htd = new HTableDescriptor(getName());
122       htd.addFamily(new HColumnDescriptor(FAMILIES[0]));
123       htd.addFamily(new HColumnDescriptor(FAMILIES[1]));
124       admin.createTable(htd);
125       HTable table = new HTable(conf, getName());
126       // Insert first half
127       for(byte [] ROW : ROWS_ONE) {
128         Put p = new Put(ROW);
129         for(byte [] QUALIFIER : QUALIFIERS_ONE) {
130           p.add(FAMILIES[0], QUALIFIER, VALUES[0]);
131         }
132         table.put(p);
133       }
134       for(byte [] ROW : ROWS_TWO) {
135         Put p = new Put(ROW);
136         for(byte [] QUALIFIER : QUALIFIERS_TWO) {
137           p.add(FAMILIES[1], QUALIFIER, VALUES[1]);
138         }
139         table.put(p);
140       }
141       
142       // Insert second half (reverse families)
143       for(byte [] ROW : ROWS_ONE) {
144         Put p = new Put(ROW);
145         for(byte [] QUALIFIER : QUALIFIERS_ONE) {
146           p.add(FAMILIES[1], QUALIFIER, VALUES[0]);
147         }
148         table.put(p);
149       }
150       for(byte [] ROW : ROWS_TWO) {
151         Put p = new Put(ROW);
152         for(byte [] QUALIFIER : QUALIFIERS_TWO) {
153           p.add(FAMILIES[0], QUALIFIER, VALUES[1]);
154         }
155         table.put(p);
156       }
157       
158       // Delete the second qualifier from all rows and families
159       for(byte [] ROW : ROWS_ONE) {
160         Delete d = new Delete(ROW);
161         d.deleteColumns(FAMILIES[0], QUALIFIERS_ONE[1]);
162         d.deleteColumns(FAMILIES[1], QUALIFIERS_ONE[1]);
163         table.delete(d);
164       }    
165       for(byte [] ROW : ROWS_TWO) {
166         Delete d = new Delete(ROW);
167         d.deleteColumns(FAMILIES[0], QUALIFIERS_TWO[1]);
168         d.deleteColumns(FAMILIES[1], QUALIFIERS_TWO[1]);
169         table.delete(d);
170       }
171       colsPerRow -= 2;
172       
173       // Delete the second rows from both groups, one column at a time
174       for(byte [] QUALIFIER : QUALIFIERS_ONE) {
175         Delete d = new Delete(ROWS_ONE[1]);
176         d.deleteColumns(FAMILIES[0], QUALIFIER);
177         d.deleteColumns(FAMILIES[1], QUALIFIER);
178         table.delete(d);
179       }
180       for(byte [] QUALIFIER : QUALIFIERS_TWO) {
181         Delete d = new Delete(ROWS_TWO[1]);
182         d.deleteColumns(FAMILIES[0], QUALIFIER);
183         d.deleteColumns(FAMILIES[1], QUALIFIER);
184         table.delete(d);
185       }
186       numRows -= 2;
187     }
188   }
189 
190   @Override
191   protected void tearDown() throws Exception {
192     client.shutdown();
193     super.tearDown();
194   }
195 
196   void verifyScan(Scan s, long expectedRows, long expectedKeys) 
197       throws Exception {
198     ScannerModel model = ScannerModel.fromScan(s);
199     model.setBatch(Integer.MAX_VALUE); // fetch it all at once
200     StringWriter writer = new StringWriter();
201     marshaller.marshal(model, writer);
202     LOG.debug(writer.toString());
203     byte[] body = Bytes.toBytes(writer.toString());
204     Response response = client.put("/" + getName() + "/scanner", MIMETYPE_XML,
205       body);
206     assertEquals(response.getCode(), 201);
207     String scannerURI = response.getLocation();
208     assertNotNull(scannerURI);
209 
210     // get a cell set
211     response = client.get(scannerURI, MIMETYPE_XML);
212     assertEquals(response.getCode(), 200);
213     CellSetModel cells = (CellSetModel)
214       unmarshaller.unmarshal(new ByteArrayInputStream(response.getBody()));
215 
216     int rows = cells.getRows().size();
217     assertTrue("Scanned too many rows! Only expected " + expectedRows + 
218         " total but scanned " + rows, expectedRows == rows);
219     for (RowModel row: cells.getRows()) {
220       int count = row.getCells().size();
221       assertEquals("Expected " + expectedKeys + " keys per row but " +
222         "returned " + count, expectedKeys, count);
223     }
224 
225     // delete the scanner
226     response = client.delete(scannerURI);
227     assertEquals(response.getCode(), 200);
228   }
229 
230   void verifyScanFull(Scan s, KeyValue [] kvs) throws Exception {
231     ScannerModel model = ScannerModel.fromScan(s);
232     model.setBatch(Integer.MAX_VALUE); // fetch it all at once
233     StringWriter writer = new StringWriter();
234     marshaller.marshal(model, writer);
235     LOG.debug(writer.toString());
236     byte[] body = Bytes.toBytes(writer.toString());
237     Response response = client.put("/" + getName() + "/scanner", MIMETYPE_XML,
238       body);
239     assertEquals(response.getCode(), 201);
240     String scannerURI = response.getLocation();
241     assertNotNull(scannerURI);
242 
243     // get a cell set
244     response = client.get(scannerURI, MIMETYPE_XML);
245     assertEquals(response.getCode(), 200);
246     CellSetModel cellSet = (CellSetModel)
247       unmarshaller.unmarshal(new ByteArrayInputStream(response.getBody()));
248 
249     // delete the scanner
250     response = client.delete(scannerURI);
251     assertEquals(response.getCode(), 200);
252 
253     int row = 0;
254     int idx = 0;
255     Iterator<RowModel> i = cellSet.getRows().iterator();
256     for (boolean done = true; done; row++) {
257       done = i.hasNext();
258       if (!done) break;
259       RowModel rowModel = i.next();
260       List<CellModel> cells = rowModel.getCells();
261       if (cells.isEmpty()) break;
262       assertTrue("Scanned too many keys! Only expected " + kvs.length + 
263         " total but already scanned " + (cells.size() + idx), 
264         kvs.length >= idx + cells.size());
265       for (CellModel cell: cells) {
266         assertTrue("Row mismatch", 
267             Bytes.equals(rowModel.getKey(), kvs[idx].getRow()));
268         byte[][] split = KeyValue.parseColumn(cell.getColumn());
269         assertTrue("Family mismatch", 
270             Bytes.equals(split[0], kvs[idx].getFamily()));
271         assertTrue("Qualifier mismatch", 
272             Bytes.equals(split[1], kvs[idx].getQualifier()));
273         assertTrue("Value mismatch", 
274             Bytes.equals(cell.getValue(), kvs[idx].getValue()));
275         idx++;
276       }
277     }
278     assertEquals("Expected " + kvs.length + " total keys but scanned " + idx,
279       kvs.length, idx);
280   }
281 
282   void verifyScanNoEarlyOut(Scan s, long expectedRows, long expectedKeys) 
283       throws Exception {
284     ScannerModel model = ScannerModel.fromScan(s);
285     model.setBatch(Integer.MAX_VALUE); // fetch it all at once
286     StringWriter writer = new StringWriter();
287     marshaller.marshal(model, writer);
288     LOG.debug(writer.toString());
289     byte[] body = Bytes.toBytes(writer.toString());
290     Response response = client.put("/" + getName() + "/scanner", MIMETYPE_XML,
291       body);
292     assertEquals(response.getCode(), 201);
293     String scannerURI = response.getLocation();
294     assertNotNull(scannerURI);
295 
296     // get a cell set
297     response = client.get(scannerURI, MIMETYPE_XML);
298     assertEquals(response.getCode(), 200);
299     CellSetModel cellSet = (CellSetModel)
300       unmarshaller.unmarshal(new ByteArrayInputStream(response.getBody()));
301 
302     // delete the scanner
303     response = client.delete(scannerURI);
304     assertEquals(response.getCode(), 200);
305 
306     Iterator<RowModel> i = cellSet.getRows().iterator();
307     int j = 0;
308     for (boolean done = true; done; j++) {
309       done = i.hasNext();
310       if (!done) break;
311       RowModel rowModel = i.next();
312       List<CellModel> cells = rowModel.getCells();
313       if (cells.isEmpty()) break;
314       assertTrue("Scanned too many rows! Only expected " + expectedRows + 
315         " total but already scanned " + (j+1), expectedRows > j);
316       assertEquals("Expected " + expectedKeys + " keys per row but " +
317         "returned " + cells.size(), expectedKeys, cells.size());
318     }
319     assertEquals("Expected " + expectedRows + " rows but scanned " + j +
320       " rows", expectedRows, j);
321   }
322 
323   void doTestNoFilter() throws Exception {
324     // No filter
325     long expectedRows = this.numRows;
326     long expectedKeys = this.colsPerRow;
327     
328     // Both families
329     Scan s = new Scan();
330     verifyScan(s, expectedRows, expectedKeys);
331 
332     // One family
333     s = new Scan();
334     s.addFamily(FAMILIES[0]);
335     verifyScan(s, expectedRows, expectedKeys/2);
336   }
337 
338   void doTestPrefixFilter() throws Exception {
339     // Grab rows from group one (half of total)
340     long expectedRows = this.numRows / 2;
341     long expectedKeys = this.colsPerRow;
342     Scan s = new Scan();
343     s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
344     verifyScan(s, expectedRows, expectedKeys);
345   }
346 
347   void doTestPageFilter() throws Exception {
348     // KVs in first 6 rows
349     KeyValue [] expectedKVs = {
350       // testRowOne-0
351       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
352       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
353       new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
354       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
355       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
356       new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
357       // testRowOne-2
358       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
359       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
360       new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
361       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
362       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
363       new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
364       // testRowOne-3
365       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
366       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
367       new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
368       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
369       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
370       new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
371       // testRowTwo-0
372       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
373       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
374       new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
375       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
376       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
377       new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
378       // testRowTwo-2
379       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
380       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
381       new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
382       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
383       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
384       new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
385       // testRowTwo-3
386       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
387       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
388       new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
389       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
390       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
391       new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
392     };
393     
394     // Grab all 6 rows
395     long expectedRows = 6;
396     long expectedKeys = this.colsPerRow;
397     Scan s = new Scan();
398     s.setFilter(new PageFilter(expectedRows));
399     verifyScan(s, expectedRows, expectedKeys);
400     s.setFilter(new PageFilter(expectedRows));
401     verifyScanFull(s, expectedKVs);
402     
403     // Grab first 4 rows (6 cols per row)
404     expectedRows = 4;
405     expectedKeys = this.colsPerRow;
406     s = new Scan();
407     s.setFilter(new PageFilter(expectedRows));
408     verifyScan(s, expectedRows, expectedKeys);
409     s.setFilter(new PageFilter(expectedRows));
410     verifyScanFull(s, Arrays.copyOf(expectedKVs, 24));
411     
412     // Grab first 2 rows
413     expectedRows = 2;
414     expectedKeys = this.colsPerRow;
415     s = new Scan();
416     s.setFilter(new PageFilter(expectedRows));
417     verifyScan(s, expectedRows, expectedKeys);
418     s.setFilter(new PageFilter(expectedRows));
419     verifyScanFull(s, Arrays.copyOf(expectedKVs, 12));
420 
421     // Grab first row
422     expectedRows = 1;
423     expectedKeys = this.colsPerRow;
424     s = new Scan();
425     s.setFilter(new PageFilter(expectedRows));
426     verifyScan(s, expectedRows, expectedKeys);
427     s.setFilter(new PageFilter(expectedRows));
428     verifyScanFull(s, Arrays.copyOf(expectedKVs, 6));    
429   }
430 
431   void doTestInclusiveStopFilter() throws Exception {
432     // Grab rows from group one
433     
434     // If we just use start/stop row, we get total/2 - 1 rows
435     long expectedRows = (this.numRows / 2) - 1;
436     long expectedKeys = this.colsPerRow;
437     Scan s = new Scan(Bytes.toBytes("testRowOne-0"), 
438         Bytes.toBytes("testRowOne-3"));
439     verifyScan(s, expectedRows, expectedKeys);
440     
441     // Now use start row with inclusive stop filter
442     expectedRows = this.numRows / 2;
443     s = new Scan(Bytes.toBytes("testRowOne-0"));
444     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowOne-3")));
445     verifyScan(s, expectedRows, expectedKeys);
446 
447     // Grab rows from group two
448     
449     // If we just use start/stop row, we get total/2 - 1 rows
450     expectedRows = (this.numRows / 2) - 1;
451     expectedKeys = this.colsPerRow;
452     s = new Scan(Bytes.toBytes("testRowTwo-0"), 
453         Bytes.toBytes("testRowTwo-3"));
454     verifyScan(s, expectedRows, expectedKeys);
455     
456     // Now use start row with inclusive stop filter
457     expectedRows = this.numRows / 2;
458     s = new Scan(Bytes.toBytes("testRowTwo-0"));
459     s.setFilter(new InclusiveStopFilter(Bytes.toBytes("testRowTwo-3")));
460     verifyScan(s, expectedRows, expectedKeys);
461 
462   }
463   
464   void doTestQualifierFilter() throws Exception {
465     // Match two keys (one from each family) in half the rows
466     long expectedRows = this.numRows / 2;
467     long expectedKeys = 2;
468     Filter f = new QualifierFilter(CompareOp.EQUAL,
469         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
470     Scan s = new Scan();
471     s.setFilter(f);
472     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
473     
474     // Match keys less than same qualifier
475     // Expect only two keys (one from each family) in half the rows
476     expectedRows = this.numRows / 2;
477     expectedKeys = 2;
478     f = new QualifierFilter(CompareOp.LESS,
479         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
480     s = new Scan();
481     s.setFilter(f);
482     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
483     
484     // Match keys less than or equal
485     // Expect four keys (two from each family) in half the rows
486     expectedRows = this.numRows / 2;
487     expectedKeys = 4;
488     f = new QualifierFilter(CompareOp.LESS_OR_EQUAL,
489         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
490     s = new Scan();
491     s.setFilter(f);
492     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
493     
494     // Match keys not equal
495     // Expect four keys (two from each family)
496     // Only look in first group of rows
497     expectedRows = this.numRows / 2;
498     expectedKeys = 4;
499     f = new QualifierFilter(CompareOp.NOT_EQUAL,
500         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
501     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
502     s.setFilter(f);
503     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
504     
505     // Match keys greater or equal
506     // Expect four keys (two from each family)
507     // Only look in first group of rows
508     expectedRows = this.numRows / 2;
509     expectedKeys = 4;
510     f = new QualifierFilter(CompareOp.GREATER_OR_EQUAL,
511         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
512     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
513     s.setFilter(f);
514     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
515     
516     // Match keys greater
517     // Expect two keys (one from each family)
518     // Only look in first group of rows
519     expectedRows = this.numRows / 2;
520     expectedKeys = 2;
521     f = new QualifierFilter(CompareOp.GREATER,
522         new BinaryComparator(Bytes.toBytes("testQualifierOne-2")));
523     s = new Scan(HConstants.EMPTY_START_ROW, Bytes.toBytes("testRowTwo"));
524     s.setFilter(f);
525     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
526     
527     // Match keys not equal to
528     // Look across rows and fully validate the keys and ordering
529     // Expect varied numbers of keys, 4 per row in group one, 6 per row in
530     // group two
531     f = new QualifierFilter(CompareOp.NOT_EQUAL,
532         new BinaryComparator(QUALIFIERS_ONE[2]));
533     s = new Scan();
534     s.setFilter(f);
535     
536     KeyValue [] kvs = {
537         // testRowOne-0
538         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
539         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
540         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
541         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
542         // testRowOne-2
543         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
544         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
545         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
546         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
547         // testRowOne-3
548         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
549         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
550         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
551         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
552         // testRowTwo-0
553         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
554         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
555         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
556         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
557         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
558         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
559         // testRowTwo-2
560         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
561         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
562         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
563         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
564         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
565         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
566         // testRowTwo-3
567         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
568         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
569         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
570         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
571         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
572         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
573     };
574     verifyScanFull(s, kvs);
575      
576     // Test across rows and groups with a regex
577     // Filter out "test*-2"
578     // Expect 4 keys per row across both groups
579     f = new QualifierFilter(CompareOp.NOT_EQUAL,
580         new RegexStringComparator("test.+-2"));
581     s = new Scan();
582     s.setFilter(f);
583     
584     kvs = new KeyValue [] {
585         // testRowOne-0
586         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
587         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
588         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
589         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
590         // testRowOne-2
591         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
592         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
593         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
594         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
595         // testRowOne-3
596         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
597         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
598         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
599         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
600         // testRowTwo-0
601         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
602         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
603         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
604         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
605         // testRowTwo-2
606         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
607         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
608         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
609         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
610         // testRowTwo-3
611         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
612         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
613         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
614         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
615     };
616     verifyScanFull(s, kvs);
617   }
618   
619   void doTestRowFilter() throws Exception {
620     // Match a single row, all keys
621     long expectedRows = 1;
622     long expectedKeys = this.colsPerRow;
623     Filter f = new RowFilter(CompareOp.EQUAL,
624         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
625     Scan s = new Scan();
626     s.setFilter(f);
627     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
628     
629     // Match a two rows, one from each group, using regex
630     expectedRows = 2;
631     expectedKeys = this.colsPerRow;
632     f = new RowFilter(CompareOp.EQUAL,
633         new RegexStringComparator("testRow.+-2"));
634     s = new Scan();
635     s.setFilter(f);
636     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
637     
638     // Match rows less than
639     // Expect all keys in one row
640     expectedRows = 1;
641     expectedKeys = this.colsPerRow;
642     f = new RowFilter(CompareOp.LESS,
643         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
644     s = new Scan();
645     s.setFilter(f);
646     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
647     
648     // Match rows less than or equal
649     // Expect all keys in two rows
650     expectedRows = 2;
651     expectedKeys = this.colsPerRow;
652     f = new RowFilter(CompareOp.LESS_OR_EQUAL,
653         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
654     s = new Scan();
655     s.setFilter(f);
656     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
657     
658     // Match rows not equal
659     // Expect all keys in all but one row
660     expectedRows = this.numRows - 1;
661     expectedKeys = this.colsPerRow;
662     f = new RowFilter(CompareOp.NOT_EQUAL,
663         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
664     s = new Scan();
665     s.setFilter(f);
666     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
667     
668     // Match keys greater or equal
669     // Expect all keys in all but one row
670     expectedRows = this.numRows - 1;
671     expectedKeys = this.colsPerRow;
672     f = new RowFilter(CompareOp.GREATER_OR_EQUAL,
673         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
674     s = new Scan();
675     s.setFilter(f);
676     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
677     
678     // Match keys greater
679     // Expect all keys in all but two rows
680     expectedRows = this.numRows - 2;
681     expectedKeys = this.colsPerRow;
682     f = new RowFilter(CompareOp.GREATER,
683         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
684     s = new Scan();
685     s.setFilter(f);
686     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
687     
688     // Match rows not equal to testRowTwo-2
689     // Look across rows and fully validate the keys and ordering
690     // Should see all keys in all rows but testRowTwo-2
691     f = new RowFilter(CompareOp.NOT_EQUAL,
692         new BinaryComparator(Bytes.toBytes("testRowOne-2")));
693     s = new Scan();
694     s.setFilter(f);
695     
696     KeyValue [] kvs = {
697         // testRowOne-0
698         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
699         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
700         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
701         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
702         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
703         new KeyValue(ROWS_ONE[0], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
704         // testRowOne-3
705         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
706         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
707         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
708         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
709         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
710         new KeyValue(ROWS_ONE[3], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
711         // testRowTwo-0
712         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
713         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
714         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
715         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
716         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
717         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
718         // testRowTwo-2
719         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
720         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
721         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
722         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
723         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
724         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
725         // testRowTwo-3
726         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
727         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
728         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
729         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
730         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
731         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
732     };
733     verifyScanFull(s, kvs);
734     
735     // Test across rows and groups with a regex
736     // Filter out everything that doesn't match "*-2"
737     // Expect all keys in two rows
738     f = new RowFilter(CompareOp.EQUAL,
739         new RegexStringComparator(".+-2"));
740     s = new Scan();
741     s.setFilter(f);
742     
743     kvs = new KeyValue [] {
744         // testRowOne-2
745         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
746         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0]),
747         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[3], VALUES[0]),
748         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[0], VALUES[0]),
749         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[2], VALUES[0]),
750         new KeyValue(ROWS_ONE[2], FAMILIES[1], QUALIFIERS_ONE[3], VALUES[0]),
751         // testRowTwo-2
752         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
753         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
754         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
755         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
756         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
757         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1])
758     };
759     verifyScanFull(s, kvs);
760   }
761   
762   void doTestValueFilter() throws Exception {
763     // Match group one rows
764     long expectedRows = this.numRows / 2;
765     long expectedKeys = this.colsPerRow;
766     Filter f = new ValueFilter(CompareOp.EQUAL,
767         new BinaryComparator(Bytes.toBytes("testValueOne")));
768     Scan s = new Scan();
769     s.setFilter(f);
770     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
771 
772     // Match group two rows
773     expectedRows = this.numRows / 2;
774     expectedKeys = this.colsPerRow;
775     f = new ValueFilter(CompareOp.EQUAL,
776         new BinaryComparator(Bytes.toBytes("testValueTwo")));
777     s = new Scan();
778     s.setFilter(f);
779     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
780     
781     // Match all values using regex
782     expectedRows = this.numRows;
783     expectedKeys = this.colsPerRow;
784     f = new ValueFilter(CompareOp.EQUAL,
785         new RegexStringComparator("testValue((One)|(Two))"));
786     s = new Scan();
787     s.setFilter(f);
788     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
789     
790     // Match values less than
791     // Expect group one rows
792     expectedRows = this.numRows / 2;
793     expectedKeys = this.colsPerRow;
794     f = new ValueFilter(CompareOp.LESS,
795         new BinaryComparator(Bytes.toBytes("testValueTwo")));
796     s = new Scan();
797     s.setFilter(f);
798     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
799     
800     // Match values less than or equal
801     // Expect all rows
802     expectedRows = this.numRows;
803     expectedKeys = this.colsPerRow;
804     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
805         new BinaryComparator(Bytes.toBytes("testValueTwo")));
806     s = new Scan();
807     s.setFilter(f);
808     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
809 
810     // Match values less than or equal
811     // Expect group one rows
812     expectedRows = this.numRows / 2;
813     expectedKeys = this.colsPerRow;
814     f = new ValueFilter(CompareOp.LESS_OR_EQUAL,
815         new BinaryComparator(Bytes.toBytes("testValueOne")));
816     s = new Scan();
817     s.setFilter(f);
818     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
819     
820     // Match values not equal
821     // Expect half the rows
822     expectedRows = this.numRows / 2;
823     expectedKeys = this.colsPerRow;
824     f = new ValueFilter(CompareOp.NOT_EQUAL,
825         new BinaryComparator(Bytes.toBytes("testValueOne")));
826     s = new Scan();
827     s.setFilter(f);
828     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
829     
830     // Match values greater or equal
831     // Expect all rows
832     expectedRows = this.numRows;
833     expectedKeys = this.colsPerRow;
834     f = new ValueFilter(CompareOp.GREATER_OR_EQUAL,
835         new BinaryComparator(Bytes.toBytes("testValueOne")));
836     s = new Scan();
837     s.setFilter(f);
838     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
839     
840     // Match values greater
841     // Expect half rows
842     expectedRows = this.numRows / 2;
843     expectedKeys = this.colsPerRow;
844     f = new ValueFilter(CompareOp.GREATER,
845         new BinaryComparator(Bytes.toBytes("testValueOne")));
846     s = new Scan();
847     s.setFilter(f);
848     verifyScanNoEarlyOut(s, expectedRows, expectedKeys);
849     
850     // Match values not equal to testValueOne
851     // Look across rows and fully validate the keys and ordering
852     // Should see all keys in all group two rows
853     f = new ValueFilter(CompareOp.NOT_EQUAL,
854         new BinaryComparator(Bytes.toBytes("testValueOne")));
855     s = new Scan();
856     s.setFilter(f);
857     
858     KeyValue [] kvs = {
859         // testRowTwo-0
860         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
861         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
862         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
863         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
864         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
865         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
866         // testRowTwo-2
867         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
868         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
869         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
870         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
871         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
872         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
873         // testRowTwo-3
874         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
875         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
876         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
877         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
878         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
879         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
880     };
881     verifyScanFull(s, kvs);
882   }
883   
884   void doTestSkipFilter() throws Exception {
885     // Test for qualifier regex: "testQualifierOne-2"
886     // Should only get rows from second group, and all keys
887     Filter f = new SkipFilter(new QualifierFilter(CompareOp.NOT_EQUAL,
888         new BinaryComparator(Bytes.toBytes("testQualifierOne-2"))));
889     Scan s = new Scan();
890     s.setFilter(f);
891     
892     KeyValue [] kvs = {
893         // testRowTwo-0
894         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
895         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
896         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
897         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
898         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
899         new KeyValue(ROWS_TWO[0], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
900         // testRowTwo-2
901         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
902         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
903         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
904         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
905         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
906         new KeyValue(ROWS_TWO[2], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
907         // testRowTwo-3
908         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
909         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[2], VALUES[1]),
910         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[3], VALUES[1]),
911         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[0], VALUES[1]),
912         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[2], VALUES[1]),
913         new KeyValue(ROWS_TWO[3], FAMILIES[1], QUALIFIERS_TWO[3], VALUES[1]),
914     };
915     verifyScanFull(s, kvs);
916   }
917     
918   void doTestFilterList() throws Exception {
919     // Test getting a single row, single key using Row, Qualifier, and Value 
920     // regular expression and substring filters
921     // Use must pass all
922     List<Filter> filters = new ArrayList<Filter>();
923     filters.add(new RowFilter(CompareOp.EQUAL,
924       new RegexStringComparator(".+-2")));
925     filters.add(new QualifierFilter(CompareOp.EQUAL,
926       new RegexStringComparator(".+-2")));
927     filters.add(new ValueFilter(CompareOp.EQUAL,
928       new SubstringComparator("One")));
929     Filter f = new FilterList(Operator.MUST_PASS_ALL, filters);
930     Scan s = new Scan();
931     s.addFamily(FAMILIES[0]);
932     s.setFilter(f);
933     KeyValue [] kvs = {
934         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[2], VALUES[0])
935     };
936     verifyScanFull(s, kvs);
937 
938     // Test getting everything with a MUST_PASS_ONE filter including row, qf,
939     // val, regular expression and substring filters
940     filters.clear();
941     filters.add(new RowFilter(CompareOp.EQUAL,
942       new RegexStringComparator(".+Two.+")));
943     filters.add(new QualifierFilter(CompareOp.EQUAL,
944       new RegexStringComparator(".+-2")));
945     filters.add(new ValueFilter(CompareOp.EQUAL,
946       new SubstringComparator("One")));
947     f = new FilterList(Operator.MUST_PASS_ONE, filters);
948     s = new Scan();
949     s.setFilter(f);
950     verifyScanNoEarlyOut(s, this.numRows, this.colsPerRow);    
951   }
952   
953   void doTestFirstKeyOnlyFilter() throws Exception {
954     Scan s = new Scan();
955     s.setFilter(new FirstKeyOnlyFilter());
956     // Expected KVs, the first KV from each of the remaining 6 rows
957     KeyValue [] kvs = {
958         new KeyValue(ROWS_ONE[0], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
959         new KeyValue(ROWS_ONE[2], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
960         new KeyValue(ROWS_ONE[3], FAMILIES[0], QUALIFIERS_ONE[0], VALUES[0]),
961         new KeyValue(ROWS_TWO[0], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
962         new KeyValue(ROWS_TWO[2], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1]),
963         new KeyValue(ROWS_TWO[3], FAMILIES[0], QUALIFIERS_TWO[0], VALUES[1])
964     };
965     verifyScanFull(s, kvs);
966   }
967   
968   public void testScannersWithFilters() throws Exception {
969     doTestNoFilter();
970     doTestPrefixFilter();
971     doTestPageFilter();
972     doTestInclusiveStopFilter();
973     doTestQualifierFilter();
974     doTestRowFilter();
975     doTestValueFilter();
976     doTestSkipFilter();
977     doTestFilterList();
978     doTestFirstKeyOnlyFilter();
979   }
980 }