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