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