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