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