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.filter;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertNull;
023import static org.junit.Assert.assertTrue;
024import static org.junit.Assert.fail;
025
026import java.io.IOException;
027import java.util.ArrayList;
028import java.util.Arrays;
029import java.util.List;
030import org.apache.hadoop.hbase.Cell;
031import org.apache.hadoop.hbase.CellComparator;
032import org.apache.hadoop.hbase.CompareOperator;
033import org.apache.hadoop.hbase.HBaseClassTestRule;
034import org.apache.hadoop.hbase.KeyValue;
035import org.apache.hadoop.hbase.KeyValueUtil;
036import org.apache.hadoop.hbase.exceptions.DeserializationException;
037import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
038import org.apache.hadoop.hbase.filter.FilterList.Operator;
039import org.apache.hadoop.hbase.testclassification.FilterTests;
040import org.apache.hadoop.hbase.testclassification.SmallTests;
041import org.apache.hadoop.hbase.util.Bytes;
042import org.junit.Assert;
043import org.junit.ClassRule;
044import org.junit.Test;
045import org.junit.experimental.categories.Category;
046import org.mockito.Mockito;
047
048import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
049
050import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
051
052@Category({FilterTests.class, SmallTests.class})
053public class TestFilterList {
054
055  @ClassRule
056  public static final HBaseClassTestRule CLASS_RULE =
057      HBaseClassTestRule.forClass(TestFilterList.class);
058
059  static final int MAX_PAGES = 2;
060
061  @Test
062  public void testAddFilter() throws Exception {
063    Filter filter1 = new FirstKeyOnlyFilter();
064    Filter filter2 = new FirstKeyOnlyFilter();
065
066    FilterList filterList = new FilterList(filter1, filter2);
067    filterList.addFilter(new FirstKeyOnlyFilter());
068
069    filterList = new FilterList(Arrays.asList(filter1, filter2));
070    filterList.addFilter(new FirstKeyOnlyFilter());
071
072    filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
073    filterList.addFilter(new FirstKeyOnlyFilter());
074
075    filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(filter1, filter2));
076    filterList.addFilter(new FirstKeyOnlyFilter());
077
078    filterList.setReversed(false);
079    FirstKeyOnlyFilter f = new FirstKeyOnlyFilter();
080    f.setReversed(true);
081    try {
082      filterList.addFilter(f);
083      fail("The IllegalArgumentException should be thrown because the added filter is reversed");
084    } catch (IllegalArgumentException e) {
085    }
086
087  }
088
089  @Test
090  public void testConstruction() {
091    FirstKeyOnlyFilter f1 = new FirstKeyOnlyFilter();
092    FirstKeyOnlyFilter f2 = new FirstKeyOnlyFilter();
093    f1.setReversed(true);
094    f2.setReversed(false);
095
096    try {
097      FilterList ff = new FilterList(f1, f2);
098      fail("The IllegalArgumentException should be thrown");
099    } catch (IllegalArgumentException e) {
100    }
101
102    try {
103      FilterList ff = new FilterList(Arrays.asList(f1, f2));
104      fail("The IllegalArgumentException should be thrown because the added filter is reversed");
105    } catch (IllegalArgumentException e) {
106    }
107
108    try {
109      FilterList ff = new FilterList(FilterList.Operator.MUST_PASS_ALL, Arrays.asList(f1, f2));
110      fail("The IllegalArgumentException should be thrown because the added filter is reversed");
111    } catch (IllegalArgumentException e) {
112    }
113
114    try {
115      FilterList ff = new FilterList(FilterList.Operator.MUST_PASS_ALL, f1, f2);
116      fail("The IllegalArgumentException should be thrown because the added filter is reversed");
117    } catch (IllegalArgumentException e) {
118    }
119  }
120  /**
121   * Test "must pass one"
122   * @throws Exception
123   */
124  @Test
125  public void testMPONE() throws Exception {
126    mpOneTest(getFilterMPONE());
127  }
128
129  private Filter getFilterMPONE() {
130    List<Filter> filters = new ArrayList<>();
131    filters.add(new PageFilter(MAX_PAGES));
132    filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
133    Filter filterMPONE =
134      new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
135    return filterMPONE;
136  }
137
138  private void mpOneTest(Filter filterMPONE) throws Exception {
139    /* Filter must do all below steps:
140     * <ul>
141     * <li>{@link #reset()}</li>
142     * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
143     * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
144     * if false, we will also call</li>
145     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
146     * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
147     * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
148     * </li>
149     * </ul>
150    */
151    filterMPONE.reset();
152    assertFalse(filterMPONE.filterAllRemaining());
153
154    /* Will pass both */
155    byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
156    for (int i = 0; i < MAX_PAGES - 1; i++) {
157      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
158      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
159        Bytes.toBytes(i));
160      assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
161      assertFalse(filterMPONE.filterRow());
162    }
163
164    /* Only pass PageFilter */
165    rowkey = Bytes.toBytes("z");
166    assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
167    KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
168        Bytes.toBytes(0));
169    assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
170    assertFalse(filterMPONE.filterRow());
171
172    /* reach MAX_PAGES already, should filter any rows */
173    rowkey = Bytes.toBytes("yyy");
174    assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
175    kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
176        Bytes.toBytes(0));
177    assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
178    assertFalse(filterMPONE.filterRow());
179
180    /* We should filter any row */
181    rowkey = Bytes.toBytes("z");
182    assertTrue(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
183    assertTrue(filterMPONE.filterAllRemaining());
184  }
185
186  /**
187   * Test "must pass all"
188   * @throws Exception
189   */
190  @Test
191  public void testMPALL() throws Exception {
192    mpAllTest(getMPALLFilter());
193  }
194
195  private Filter getMPALLFilter() {
196    List<Filter> filters = new ArrayList<>();
197    filters.add(new PageFilter(MAX_PAGES));
198    filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
199    Filter filterMPALL =
200      new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
201    return filterMPALL;
202  }
203
204  private void mpAllTest(Filter filterMPALL) throws Exception {
205    /* Filter must do all below steps:
206     * <ul>
207     * <li>{@link #reset()}</li>
208     * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
209     * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
210     * if false, we will also call</li>
211     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this cell</li>
212     * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
213     * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
214     * </li>
215     * </ul>
216    */
217    filterMPALL.reset();
218    assertFalse(filterMPALL.filterAllRemaining());
219    byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
220    for (int i = 0; i < MAX_PAGES - 1; i++) {
221      assertFalse(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
222      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
223        Bytes.toBytes(i));
224      assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterCell(kv));
225    }
226    filterMPALL.reset();
227    rowkey = Bytes.toBytes("z");
228    assertTrue(filterMPALL.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
229    // Should fail here; row should be filtered out.
230    KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
231    assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterCell(kv));
232  }
233
234  /**
235   * Test list ordering
236   * @throws Exception
237   */
238  @Test
239  public void testOrdering() throws Exception {
240    orderingTest(getOrderingFilter());
241  }
242
243  public Filter getOrderingFilter() {
244    List<Filter> filters = new ArrayList<>();
245    filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
246    filters.add(new PageFilter(MAX_PAGES));
247    Filter filterMPONE =
248      new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
249    return filterMPONE;
250  }
251
252  public void orderingTest(Filter filterMPONE) throws Exception {
253    /* Filter must do all below steps:
254     * <ul>
255     * <li>{@link #reset()}</li>
256     * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
257     * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
258     * if false, we will also call</li>
259     * <li>{@link #filterCell(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
260     * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
261     * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
262     * </li>
263     * </ul>
264    */
265    filterMPONE.reset();
266    assertFalse(filterMPONE.filterAllRemaining());
267
268    /* We should be able to fill MAX_PAGES without incrementing page counter */
269    byte [] rowkey = Bytes.toBytes("yyyyyyyy");
270    for (int i = 0; i < MAX_PAGES; i++) {
271      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
272      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
273          Bytes.toBytes(i));
274        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
275      assertFalse(filterMPONE.filterRow());
276    }
277
278    /* Now let's fill the page filter */
279    rowkey = Bytes.toBytes("xxxxxxx");
280    for (int i = 0; i < MAX_PAGES; i++) {
281      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
282      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
283          Bytes.toBytes(i));
284        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
285      assertFalse(filterMPONE.filterRow());
286    }
287
288    /* We should still be able to include even though page filter is at max */
289    rowkey = Bytes.toBytes("yyy");
290    for (int i = 0; i < MAX_PAGES; i++) {
291      assertFalse(filterMPONE.filterRowKey(KeyValueUtil.createFirstOnRow(rowkey)));
292      KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
293          Bytes.toBytes(i));
294        assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterCell(kv));
295      assertFalse(filterMPONE.filterRow());
296    }
297  }
298
299  /**
300   * When we do a "MUST_PASS_ONE" (a logical 'OR') of the above two filters
301   * we expect to get the same result as the 'prefix' only result.
302   * @throws Exception
303   */
304  @Test
305  public void testFilterListTwoFiltersMustPassOne() throws Exception {
306    byte[] r1 = Bytes.toBytes("Row1");
307    byte[] r11 = Bytes.toBytes("Row11");
308    byte[] r2 = Bytes.toBytes("Row2");
309
310    FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
311    flist.addFilter(new PrefixFilter(r1));
312    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
313    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
314    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
315
316    flist.reset();
317    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
318    assertEquals(ReturnCode.SKIP, flist.filterCell(new KeyValue(r2, r2, r2)));
319
320    flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
321    flist.addFilter(new AlwaysNextColFilter());
322    flist.addFilter(new PrefixFilter(r1));
323    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
324    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
325    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
326
327    flist.reset();
328    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
329    assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
330  }
331
332  /**
333   * When we do a "MUST_PASS_ONE" (a logical 'OR') of the two filters
334   * we expect to get the same result as the inclusive stop result.
335   * @throws Exception
336   */
337  @Test
338  public void testFilterListWithInclusiveStopFilterMustPassOne() throws Exception {
339    byte[] r1 = Bytes.toBytes("Row1");
340    byte[] r11 = Bytes.toBytes("Row11");
341    byte[] r2 = Bytes.toBytes("Row2");
342
343    FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
344    flist.addFilter(new AlwaysNextColFilter());
345    flist.addFilter(new InclusiveStopFilter(r1));
346    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
347    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r1, r1, r1)));
348    assertEquals(ReturnCode.INCLUDE, flist.filterCell(new KeyValue(r11, r11, r11)));
349
350    flist.reset();
351    flist.filterRowKey(KeyValueUtil.createFirstOnRow(r2));
352    assertEquals(ReturnCode.NEXT_COL, flist.filterCell(new KeyValue(r2, r2, r2)));
353  }
354
355  public static class AlwaysNextColFilter extends FilterBase {
356    public AlwaysNextColFilter() {
357      super();
358    }
359
360    @Override
361    public ReturnCode filterCell(final Cell v) {
362      return ReturnCode.NEXT_COL;
363    }
364
365    public static AlwaysNextColFilter parseFrom(final byte[] pbBytes)
366        throws DeserializationException {
367      return new AlwaysNextColFilter();
368    }
369  }
370
371  /**
372   * Test serialization
373   * @throws Exception
374   */
375  @Test
376  public void testSerialization() throws Exception {
377    List<Filter> filters = new ArrayList<>();
378    filters.add(new PageFilter(MAX_PAGES));
379    filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
380    Filter filterMPALL =
381      new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
382
383    // Decompose filterMPALL to bytes.
384    byte[] buffer = filterMPALL.toByteArray();
385
386    // Recompose filterMPALL.
387    FilterList newFilter = FilterList.parseFrom(buffer);
388
389    // Run tests
390    mpOneTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getFilterMPONE())));
391    mpAllTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getMPALLFilter())));
392    orderingTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getOrderingFilter())));
393  }
394
395  /**
396   * Test filterCell logic.
397   * @throws Exception
398   */
399  @Test
400  public void testFilterCell() throws Exception {
401    Filter includeFilter = new FilterBase() {
402      @Override
403      public Filter.ReturnCode filterCell(final Cell v) {
404        return Filter.ReturnCode.INCLUDE;
405      }
406    };
407
408    Filter alternateFilter = new FilterBase() {
409      boolean returnInclude = true;
410
411      @Override
412      public Filter.ReturnCode filterCell(final Cell v) {
413        Filter.ReturnCode returnCode = returnInclude ? Filter.ReturnCode.INCLUDE :
414                                                       Filter.ReturnCode.SKIP;
415        returnInclude = !returnInclude;
416        return returnCode;
417      }
418    };
419
420    Filter alternateIncludeFilter = new FilterBase() {
421      boolean returnIncludeOnly = false;
422
423      @Override
424      public Filter.ReturnCode filterCell(final Cell v) {
425        Filter.ReturnCode returnCode = returnIncludeOnly ? Filter.ReturnCode.INCLUDE :
426                                                           Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
427        returnIncludeOnly = !returnIncludeOnly;
428        return returnCode;
429      }
430    };
431
432    // Check must pass one filter.
433    FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
434        Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
435    // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
436    assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
437    // INCLUDE, SKIP, INCLUDE.
438    assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterCell(null));
439
440    // Check must pass all filter.
441    FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
442        Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
443    // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
444    assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterCell(null));
445    // INCLUDE, SKIP, INCLUDE.
446    assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterCell(null));
447  }
448
449  /**
450   * Test pass-thru of hints.
451   */
452  @Test
453  public void testHintPassThru() throws Exception {
454
455    final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
456    final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE),
457        null, null);
458
459    Filter filterNoHint = new FilterBase() {
460      @Override
461      public byte [] toByteArray() {
462        return null;
463      }
464
465      @Override
466      public ReturnCode filterCell(final Cell ignored) throws IOException {
467        return ReturnCode.INCLUDE;
468      }
469    };
470
471    Filter filterMinHint = new FilterBase() {
472      @Override
473      public ReturnCode filterCell(final Cell ignored) {
474        return ReturnCode.SEEK_NEXT_USING_HINT;
475      }
476
477      @Override
478      public Cell getNextCellHint(Cell currentKV) {
479        return minKeyValue;
480      }
481
482      @Override
483      public byte [] toByteArray() {return null;}
484    };
485
486    Filter filterMaxHint = new FilterBase() {
487      @Override
488      public ReturnCode filterCell(final Cell ignored) {
489        return ReturnCode.SEEK_NEXT_USING_HINT;
490      }
491
492      @Override
493      public Cell getNextCellHint(Cell cell) {
494        return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
495      }
496
497      @Override
498      public byte [] toByteArray() {return null;}
499    };
500
501    CellComparator comparator = CellComparator.getInstance();
502    // MUST PASS ONE
503
504    // Should take the min if given two hints
505    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
506        Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
507    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
508
509    // Should have no hint if any filter has no hint
510    filterList = new FilterList(Operator.MUST_PASS_ONE,
511        Arrays.asList(
512            new Filter [] { filterMinHint, filterMaxHint, filterNoHint } ));
513    assertNull(filterList.getNextCellHint(null));
514    filterList = new FilterList(Operator.MUST_PASS_ONE,
515        Arrays.asList(new Filter [] { filterNoHint, filterMaxHint } ));
516    assertNull(filterList.getNextCellHint(null));
517
518    // Should give max hint if its the only one
519    filterList = new FilterList(Operator.MUST_PASS_ONE,
520        Arrays.asList(new Filter[] { filterMaxHint, filterMaxHint }));
521    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), maxKeyValue));
522
523    // MUST PASS ALL
524
525    // Should take the first hint
526    filterList = new FilterList(Operator.MUST_PASS_ALL,
527        Arrays.asList(new Filter [] { filterMinHint, filterMaxHint } ));
528    filterList.filterCell(null);
529    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
530
531    filterList = new FilterList(Operator.MUST_PASS_ALL,
532        Arrays.asList(new Filter [] { filterMaxHint, filterMinHint } ));
533    filterList.filterCell(null);
534    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), maxKeyValue));
535
536    // Should have first hint even if a filter has no hint
537    filterList = new FilterList(Operator.MUST_PASS_ALL,
538        Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
539    filterList.filterCell(null);
540    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
541    filterList = new FilterList(Operator.MUST_PASS_ALL,
542        Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
543    filterList.filterCell(null);
544    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), maxKeyValue));
545    filterList = new FilterList(Operator.MUST_PASS_ALL,
546        Arrays.asList(new Filter[] { filterNoHint, filterMinHint }));
547    filterList.filterCell(null);
548    assertEquals(0, comparator.compare(filterList.getNextCellHint(null), minKeyValue));
549  }
550
551  /**
552   * Tests the behavior of transform() in a hierarchical filter.
553   *
554   * transform() only applies after a filterCell() whose return-code includes the KeyValue.
555   * Lazy evaluation of AND
556   */
557  @Test
558  public void testTransformMPO() throws Exception {
559    // Apply the following filter:
560    //     (family=fam AND qualifier=qual1 AND KeyOnlyFilter)
561    //  OR (family=fam AND qualifier=qual2)
562    final FilterList flist = new FilterList(Operator.MUST_PASS_ONE, Lists.<Filter>newArrayList(
563        new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
564            new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
565            new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("qual1"))),
566            new KeyOnlyFilter())),
567        new FilterList(Operator.MUST_PASS_ALL, Lists.<Filter>newArrayList(
568            new FamilyFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
569            new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("qual2")))))));
570
571    final KeyValue kvQual1 = new KeyValue(
572        Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual1"), Bytes.toBytes("value"));
573    final KeyValue kvQual2 = new KeyValue(
574        Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual2"), Bytes.toBytes("value"));
575    final KeyValue kvQual3 = new KeyValue(
576        Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual3"), Bytes.toBytes("value"));
577
578    // Value for fam:qual1 should be stripped:
579    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual1));
580    final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1));
581    assertEquals(0, transformedQual1.getValueLength());
582
583    // Value for fam:qual2 should not be stripped:
584    assertEquals(Filter.ReturnCode.INCLUDE, flist.filterCell(kvQual2));
585    final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2));
586    assertEquals("value", Bytes.toString(transformedQual2.getValueArray(),
587      transformedQual2.getValueOffset(), transformedQual2.getValueLength()));
588
589    // Other keys should be skipped:
590    assertEquals(Filter.ReturnCode.SKIP, flist.filterCell(kvQual3));
591  }
592
593  @Test
594  public void testWithMultiVersionsInSameRow() throws Exception {
595    FilterList filterList01 =
596        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 0));
597
598    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
599        1, Bytes.toBytes("value"));
600    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
601        2, Bytes.toBytes("value"));
602    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
603        3, Bytes.toBytes("value"));
604
605    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterCell(kv1));
606    assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv2));
607    assertEquals(ReturnCode.NEXT_COL, filterList01.filterCell(kv3));
608
609    FilterList filterList11 =
610        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
611
612    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv1));
613    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv2));
614    assertEquals(ReturnCode.NEXT_COL, filterList11.filterCell(kv3));
615  }
616
617  @Test
618  public void testMPONEWithSeekNextUsingHint() throws Exception {
619    byte[] col = Bytes.toBytes("c");
620    FilterList filterList =
621        new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, col));
622
623    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
624        Bytes.toBytes("value"));
625    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 2,
626        Bytes.toBytes("value"));
627    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 3,
628        Bytes.toBytes("value"));
629    KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
630        Bytes.toBytes("value"));
631
632    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
633    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv2));
634    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
635    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv4));
636  }
637
638  private static class MockFilter extends FilterBase {
639    private ReturnCode targetRetCode;
640    public boolean didCellPassToTheFilter = false;
641
642    public MockFilter(ReturnCode targetRetCode) {
643      this.targetRetCode = targetRetCode;
644    }
645
646    @Override
647    public ReturnCode filterCell(final Cell v) throws IOException {
648      this.didCellPassToTheFilter = true;
649      return targetRetCode;
650    }
651  }
652
653  @Test
654  public void testShouldPassCurrentCellToFilter() throws IOException {
655    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
656        Bytes.toBytes("value"));
657    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 2,
658        Bytes.toBytes("value"));
659    KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 3,
660        Bytes.toBytes("value"));
661    KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
662        Bytes.toBytes("value"));
663
664    MockFilter mockFilter = new MockFilter(ReturnCode.NEXT_COL);
665    FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
666
667    filter.filterCell(kv1);
668    assertTrue(mockFilter.didCellPassToTheFilter);
669
670    mockFilter.didCellPassToTheFilter = false;
671    filter.filterCell(kv2);
672    assertFalse(mockFilter.didCellPassToTheFilter);
673
674    mockFilter.didCellPassToTheFilter = false;
675    filter.filterCell(kv3);
676    assertTrue(mockFilter.didCellPassToTheFilter);
677
678    mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
679    filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
680
681    filter.filterCell(kv1);
682    assertTrue(mockFilter.didCellPassToTheFilter);
683
684    mockFilter.didCellPassToTheFilter = false;
685    filter.filterCell(kv2);
686    assertFalse(mockFilter.didCellPassToTheFilter);
687
688    mockFilter.didCellPassToTheFilter = false;
689    filter.filterCell(kv3);
690    assertTrue(mockFilter.didCellPassToTheFilter);
691
692    mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
693    filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
694    filter.filterCell(kv1);
695    assertTrue(mockFilter.didCellPassToTheFilter);
696
697    mockFilter.didCellPassToTheFilter = false;
698    filter.filterCell(kv2);
699    assertFalse(mockFilter.didCellPassToTheFilter);
700
701    mockFilter.didCellPassToTheFilter = false;
702    filter.filterCell(kv3);
703    assertFalse(mockFilter.didCellPassToTheFilter);
704
705    filter.reset();
706    mockFilter.didCellPassToTheFilter = false;
707    filter.filterCell(kv4);
708    assertTrue(mockFilter.didCellPassToTheFilter);
709
710    mockFilter = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
711    filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
712    filter.filterCell(kv1);
713    assertTrue(mockFilter.didCellPassToTheFilter);
714
715    mockFilter.didCellPassToTheFilter = false;
716    filter.filterCell(kv2);
717    assertFalse(mockFilter.didCellPassToTheFilter);
718
719    mockFilter.didCellPassToTheFilter = false;
720    filter.filterCell(kv3);
721    assertFalse(mockFilter.didCellPassToTheFilter);
722
723    filter.reset();
724    mockFilter.didCellPassToTheFilter = false;
725    filter.filterCell(kv4);
726    assertTrue(mockFilter.didCellPassToTheFilter);
727  }
728
729  @Test
730  public void testTheMaximalRule() throws IOException {
731    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
732        Bytes.toBytes("value"));
733    MockFilter filter1 = new MockFilter(ReturnCode.INCLUDE);
734    MockFilter filter2 = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
735    MockFilter filter3 = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
736    MockFilter filter4 = new MockFilter(ReturnCode.NEXT_COL);
737    MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
738    MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
739    MockFilter filter7 = new MockFilter(ReturnCode.NEXT_ROW);
740
741    FilterList filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
742    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
743
744    filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3);
745    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
746
747    filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
748    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1));
749
750    filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
751    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv1));
752
753    filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
754    assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv1));
755
756    filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5);
757    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
758
759    filterList = new FilterList(Operator.MUST_PASS_ALL, filter2,
760        new FilterList(Operator.MUST_PASS_ALL, filter3, filter4));
761    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
762
763    filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7);
764    assertEquals(ReturnCode.NEXT_ROW, filterList.filterCell(kv1));
765  }
766
767  @Test
768  public void testTheMinimalRule() throws IOException {
769    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
770        Bytes.toBytes("value"));
771    MockFilter filter1 = new MockFilter(ReturnCode.INCLUDE);
772    MockFilter filter2 = new MockFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
773    MockFilter filter3 = new MockFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
774    MockFilter filter4 = new MockFilter(ReturnCode.NEXT_COL);
775    MockFilter filter5 = new MockFilter(ReturnCode.SKIP);
776    MockFilter filter6 = new MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
777    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
778    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
779
780    filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3);
781    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
782
783    filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6);
784    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
785
786    filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6);
787    assertEquals(ReturnCode.SKIP, filterList.filterCell(kv1));
788
789    filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1);
790    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
791
792    filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5);
793    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
794
795    filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
796        new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
797    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
798
799    filterList = new FilterList(Operator.MUST_PASS_ONE, filter2,
800        new FilterList(Operator.MUST_PASS_ONE, filter3, filter4));
801    assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv1));
802
803    filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
804    assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
805  }
806
807  static class MockSeekHintFilter extends FilterBase {
808    private Cell returnCell;
809
810    public MockSeekHintFilter(Cell returnCell) {
811      this.returnCell = returnCell;
812    }
813
814    @Override
815    public ReturnCode filterCell(final Cell v) throws IOException {
816      return ReturnCode.SEEK_NEXT_USING_HINT;
817    }
818
819    @Override
820    public Cell getNextCellHint(Cell currentCell) throws IOException {
821      return this.returnCell;
822    }
823  }
824
825  @Test
826  public void testReversedFilterListWithMockSeekHintFilter() throws IOException {
827    KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
828        Bytes.toBytes("value"));
829    KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
830        Bytes.toBytes("value"));
831    KeyValue kv3 = new KeyValue(Bytes.toBytes("row3"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
832        Bytes.toBytes("value"));
833    Filter filter1 = new MockSeekHintFilter(kv1);
834    filter1.setReversed(true);
835    Filter filter2 = new MockSeekHintFilter(kv2);
836    filter2.setReversed(true);
837    Filter filter3 = new MockSeekHintFilter(kv3);
838    filter3.setReversed(true);
839
840    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE);
841    filterList.setReversed(true);
842    filterList.addFilter(filter1);
843    filterList.addFilter(filter2);
844    filterList.addFilter(filter3);
845
846    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
847    Assert.assertEquals(kv3, filterList.getNextCellHint(kv1));
848
849    filterList = new FilterList(Operator.MUST_PASS_ALL);
850    filterList.setReversed(true);
851    filterList.addFilter(filter1);
852    filterList.addFilter(filter2);
853    filterList.addFilter(filter3);
854
855    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
856    Assert.assertEquals(kv1, filterList.getNextCellHint(kv1));
857  }
858
859  @Test
860  public void testReversedFilterListWithOR() throws IOException {
861    byte[] r22 = Bytes.toBytes("Row22");
862    byte[] r2 = Bytes.toBytes("Row2");
863    byte[] r1 = Bytes.toBytes("Row1");
864
865    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
866    filterList.setReversed(true);
867    PrefixFilter prefixFilter = new PrefixFilter(r2);
868    prefixFilter.setReversed(true);
869    filterList.addFilter(prefixFilter);
870    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
871    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
872    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
873
874    filterList.reset();
875    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
876    assertEquals(ReturnCode.SKIP, filterList.filterCell(new KeyValue(r1, r1, r1)));
877
878    filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
879    filterList.setReversed(true);
880    AlwaysNextColFilter alwaysNextColFilter = new AlwaysNextColFilter();
881    alwaysNextColFilter.setReversed(true);
882    prefixFilter = new PrefixFilter(r2);
883    prefixFilter.setReversed(true);
884    filterList.addFilter(alwaysNextColFilter);
885    filterList.addFilter(prefixFilter);
886    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
887    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r22, r22, r22)));
888    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(new KeyValue(r2, r2, r2)));
889
890    filterList.reset();
891    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
892    assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(new KeyValue(r1, r1, r1)));
893  }
894
895  @Test
896  public void testKeyOnlyFilterTransformCell() throws IOException {
897    Cell c;
898    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
899        1, Bytes.toBytes("value1"));
900    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
901        2, Bytes.toBytes("value2"));
902
903    Filter filter1 = new SingleColumnValueFilter(Bytes.toBytes("cf"), Bytes.toBytes("column1"),
904        CompareOperator.EQUAL, Bytes.toBytes("value1"));
905    Filter filter2 = new SingleColumnValueFilter(Bytes.toBytes("cf"), Bytes.toBytes("column1"),
906        CompareOperator.EQUAL, Bytes.toBytes("value2"));
907    FilterList internalFilterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
908
909    FilterList keyOnlyFilterFirst =
910        new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
911
912    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv1));
913    c = keyOnlyFilterFirst.transformCell(kv1);
914    assertEquals(0, c.getValueLength());
915    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterCell(kv2));
916    c = keyOnlyFilterFirst.transformCell(kv2);
917    assertEquals(0, c.getValueLength());
918
919    internalFilterList.reset();
920    FilterList keyOnlyFilterLast =
921        new FilterList(Operator.MUST_PASS_ALL, new KeyOnlyFilter(), internalFilterList);
922    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv1));
923    c = keyOnlyFilterLast.transformCell(kv1);
924    assertEquals(0, c.getValueLength());
925    assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterCell(kv2));
926    c = keyOnlyFilterLast.transformCell(kv2);
927    assertEquals(0, c.getValueLength());
928  }
929
930  @Test
931  public void testEmptyFilterListTransformCell() throws IOException {
932    KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
933        1, Bytes.toBytes("value"));
934    FilterList filterList = new FilterList(Operator.MUST_PASS_ALL);
935    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
936    assertEquals(kv, filterList.transformCell(kv));
937
938    filterList = new FilterList(Operator.MUST_PASS_ONE);
939    assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
940    assertEquals(kv, filterList.transformCell(kv));
941  }
942
943  private static class MockNextRowFilter extends FilterBase {
944    private int hitCount = 0;
945
946    @Override
947    public ReturnCode filterCell(final Cell v) throws IOException {
948      hitCount++;
949      return ReturnCode.NEXT_ROW;
950    }
951
952    public int getHitCount() {
953      return hitCount;
954    }
955  }
956
957  @Test
958  public void testRowCountFilter() throws IOException {
959    KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam1"), Bytes.toBytes("a"), 1,
960        Bytes.toBytes("value"));
961    KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam2"), Bytes.toBytes("a"), 2,
962        Bytes.toBytes("value"));
963    MockNextRowFilter mockNextRowFilter = new MockNextRowFilter();
964    FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter);
965    filter.filterCell(kv1);
966    filter.filterCell(kv2);
967    assertEquals(2, mockNextRowFilter.getHitCount());
968  }
969
970  private static class TransformFilter extends FilterBase {
971    private ReturnCode targetRetCode;
972    private boolean transformed = false;
973
974    public TransformFilter(ReturnCode targetRetCode) {
975      this.targetRetCode = targetRetCode;
976    }
977
978    @Override
979    public ReturnCode filterCell(final Cell v) throws IOException {
980      return targetRetCode;
981    }
982
983    @Override
984    public Cell transformCell(Cell c) throws IOException {
985      transformed = true;
986      return super.transformCell(c);
987    }
988
989    public boolean getTransformed() {
990      return this.transformed;
991    }
992  }
993
994  @Test
995  public void testTransformCell() throws IOException {
996    KeyValue kv =
997        new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"), 1,
998            Bytes.toBytes("value"));
999
1000    // case MUST_PASS_ONE
1001    TransformFilter filter1 = new TransformFilter(ReturnCode.INCLUDE);
1002    TransformFilter filter2 = new TransformFilter(ReturnCode.NEXT_ROW);
1003    TransformFilter filter3 = new TransformFilter(ReturnCode.SEEK_NEXT_USING_HINT);
1004    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2, filter3);
1005    Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv));
1006    Assert.assertEquals(kv, filterList.transformCell(kv));
1007    Assert.assertEquals(true, filter1.getTransformed());
1008    Assert.assertEquals(false, filter2.getTransformed());
1009    Assert.assertEquals(false, filter3.getTransformed());
1010
1011    // case MUST_PASS_ALL
1012    filter1 = new TransformFilter(ReturnCode.INCLUDE);
1013    filter2 = new TransformFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
1014    filter3 = new TransformFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
1015    filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2, filter3);
1016
1017    Assert.assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterCell(kv));
1018    Assert.assertEquals(kv, filterList.transformCell(kv));
1019    Assert.assertEquals(true, filter1.getTransformed());
1020    Assert.assertEquals(true, filter2.getTransformed());
1021    Assert.assertEquals(true, filter3.getTransformed());
1022  }
1023
1024  @Test
1025  public void testFilterListWithORWhenPassingCellMismatchPreviousRC() throws IOException {
1026    // Mainly test FilterListWithOR#calculateReturnCodeByPrevCellAndRC method with two sub-filters.
1027    KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"),
1028        100, Bytes.toBytes("value"));
1029    KeyValue kv2 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 99,
1030        Bytes.toBytes("value"));
1031    KeyValue kv3 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 1,
1032        Bytes.toBytes("value"));
1033    KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fan"), Bytes.toBytes("a"), 1,
1034        Bytes.toBytes("value"));
1035    Filter subFilter1 = Mockito.mock(FilterBase.class);
1036    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1037    Mockito.when(subFilter1.filterCell(kv2)).thenReturn(ReturnCode.NEXT_COL);
1038    Mockito.when(subFilter1.filterCell(kv3)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1039    Mockito.when(subFilter1.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1040
1041    Filter subFilter2 = Mockito.mock(FilterBase.class);
1042    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SKIP);
1043    Mockito.when(subFilter2.filterCell(kv2)).thenReturn(ReturnCode.NEXT_ROW);
1044    Mockito.when(subFilter2.filterCell(kv3)).thenReturn(ReturnCode.NEXT_ROW);
1045    Mockito.when(subFilter2.filterCell(kv4)).thenReturn(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
1046
1047    Filter filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1048    Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterCell(kv1));
1049    Assert.assertEquals(ReturnCode.NEXT_COL, filterList.filterCell(kv2));
1050    Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv3));
1051    Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterCell(kv4));
1052
1053    // One sub-filter will filterAllRemaining but other sub-filter will return SEEK_HINT
1054    subFilter1 = Mockito.mock(FilterBase.class);
1055    Mockito.when(subFilter1.filterAllRemaining()).thenReturn(true);
1056    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.NEXT_ROW);
1057
1058    subFilter2 = Mockito.mock(FilterBase.class);
1059    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1060    filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1061    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
1062
1063    // Two sub-filter returns SEEK_NEXT_USING_HINT, then we should return SEEK_NEXT_USING_HINT.
1064    subFilter1 = Mockito.mock(FilterBase.class);
1065    Mockito.when(subFilter1.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1066
1067    subFilter2 = Mockito.mock(FilterBase.class);
1068    Mockito.when(subFilter2.filterCell(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1069    filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1070    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterCell(kv1));
1071  }
1072}
1073