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