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