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