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