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