View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.filter;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertNull;
24  import static org.junit.Assert.assertTrue;
25  import static org.junit.Assert.fail;
26  
27  import com.google.common.collect.Lists;
28  
29  import java.io.IOException;
30  import java.util.ArrayList;
31  import java.util.Arrays;
32  import java.util.List;
33  import java.util.Objects;
34  
35  import org.apache.hadoop.hbase.Cell;
36  import org.apache.hadoop.hbase.KeyValue;
37  import org.apache.hadoop.hbase.KeyValueUtil;
38  import org.apache.hadoop.hbase.exceptions.DeserializationException;
39  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
40  import org.apache.hadoop.hbase.filter.Filter.ReturnCode;
41  import org.apache.hadoop.hbase.filter.FilterList.Operator;
42  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
43  import org.apache.hadoop.hbase.testclassification.FilterTests;
44  import org.apache.hadoop.hbase.testclassification.SmallTests;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.junit.Assert;
47  import org.junit.Test;
48  import org.junit.experimental.categories.Category;
49  import org.mockito.Mockito;
50  
51  @Category({FilterTests.class, SmallTests.class})
52  public class TestFilterList {
53    static final int MAX_PAGES = 2;
54  
55    @Test
56    public void testAddFilter() throws Exception {
57      Filter filter1 = new FirstKeyOnlyFilter();
58      Filter filter2 = new FirstKeyOnlyFilter();
59  
60      FilterList filterList = new FilterList(filter1, filter2);
61      filterList.addFilter(new FirstKeyOnlyFilter());
62  
63      filterList = new FilterList(Arrays.asList(filter1, filter2));
64      filterList.addFilter(new FirstKeyOnlyFilter());
65  
66      filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2);
67      filterList.addFilter(new FirstKeyOnlyFilter());
68  
69      filterList = new FilterList(Operator.MUST_PASS_ALL, Arrays.asList(filter1, filter2));
70      filterList.addFilter(new FirstKeyOnlyFilter());
71  
72      filterList.setReversed(false);
73      FirstKeyOnlyFilter f = new FirstKeyOnlyFilter();
74      f.setReversed(true);
75      try {
76        filterList.addFilter(f);
77        fail("The IllegalArgumentException should be thrown because the added filter is reversed");
78      } catch (IllegalArgumentException e) {
79      }
80  
81    }
82  
83    @Test
84    public void testConstruction() {
85      FirstKeyOnlyFilter f1 = new FirstKeyOnlyFilter();
86      FirstKeyOnlyFilter f2 = new FirstKeyOnlyFilter();
87      f1.setReversed(true);
88      f2.setReversed(false);
89  
90      try {
91        FilterList ff = new FilterList(f1, f2);
92        fail("The IllegalArgumentException should be thrown");
93      } catch (IllegalArgumentException e) {
94      }
95  
96      try {
97        FilterList ff = new FilterList(Arrays.asList((Filter) f1, (Filter) f2));
98        fail("The IllegalArgumentException should be thrown because the added filter is reversed");
99      } catch (IllegalArgumentException e) {
100     }
101 
102     try {
103       FilterList ff = new FilterList(FilterList.Operator.MUST_PASS_ALL,
104           Arrays.asList((Filter) f1, (Filter) 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, f1, f2);
111       fail("The IllegalArgumentException should be thrown because the added filter is reversed");
112     } catch (IllegalArgumentException e) {
113     }
114   }
115   /**
116    * Test "must pass one"
117    * @throws Exception
118    */
119   @Test
120   public void testMPONE() throws Exception {
121     mpOneTest(getFilterMPONE());
122   }
123 
124   private Filter getFilterMPONE() {
125     List<Filter> filters = new ArrayList<Filter>();
126     filters.add(new PageFilter(MAX_PAGES));
127     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
128     return new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
129   }
130 
131   private void mpOneTest(Filter filterMPONE) throws Exception {
132     /* Filter must do all below steps:
133      * <ul>
134      * <li>{@link #reset()}</li>
135      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
136      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
137      * if false, we will also call</li>
138      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
139      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
140      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
141      * </li>
142      * </ul>
143     */
144     filterMPONE.reset();
145     assertFalse(filterMPONE.filterAllRemaining());
146 
147     /* Will pass both */
148     byte[] rowkey = Bytes.toBytes("yyyyyyyyy");
149     for (int i = 0; i < MAX_PAGES - 1; i++) {
150       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
151       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
152       assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
153       assertFalse(filterMPONE.filterRow());
154     }
155 
156     /* Only pass PageFilter */
157     rowkey = Bytes.toBytes("z");
158     assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
159     KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0));
160     assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
161     assertFalse(filterMPONE.filterRow());
162 
163     /* reach MAX_PAGES already, should filter any rows */
164     rowkey = Bytes.toBytes("yyy");
165     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
166     kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0), Bytes.toBytes(0));
167     assertFalse(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
168     assertFalse(filterMPONE.filterRow());
169 
170     /* We should filter any row */
171     rowkey = Bytes.toBytes("z");
172     assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
173     assertTrue(filterMPONE.filterAllRemaining());
174   }
175 
176   /**
177    * Test "must pass all"
178    * @throws Exception
179    */
180   @Test
181   public void testMPALL() throws Exception {
182     mpAllTest(getMPALLFilter());
183   }
184 
185   private Filter getMPALLFilter() {
186     List<Filter> filters = new ArrayList<Filter>();
187     filters.add(new PageFilter(MAX_PAGES));
188     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
189     Filter filterMPALL = new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
190     return filterMPALL;
191   }
192 
193   private void mpAllTest(Filter filterMPALL) throws Exception {
194     /* Filter must do all below steps:
195      * <ul>
196      * <li>{@link #reset()}</li>
197      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
198      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
199      * if false, we will also call</li>
200      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</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.
203      * </li>
204      * </ul>
205     */
206     filterMPALL.reset();
207     assertFalse(filterMPALL.filterAllRemaining());
208     byte[] rowkey = Bytes.toBytes("yyyyyyyyy");
209     for (int i = 0; i < MAX_PAGES - 1; i++) {
210       assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
211       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
212       assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
213     }
214     filterMPALL.reset();
215     rowkey = Bytes.toBytes("z");
216     assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
217     // Should fail here; row should be filtered out.
218     KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
219     assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
220   }
221 
222   /**
223    * Test list ordering
224    * @throws Exception
225    */
226   @Test
227   public void testOrdering() throws Exception {
228     orderingTest(getOrderingFilter());
229   }
230 
231   public Filter getOrderingFilter() {
232     List<Filter> filters = new ArrayList<Filter>();
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     /* Filter must do all below steps:
241      * <ul>
242      * <li>{@link #reset()}</li>
243      * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
244      * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
245      * if false, we will also call</li>
246      * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
247      * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
248      * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
249      * </li>
250      * </ul>
251     */
252     filterMPONE.reset();
253     assertFalse(filterMPONE.filterAllRemaining());
254 
255     /* We should be able to fill MAX_PAGES without incrementing page counter */
256     byte[] rowkey = Bytes.toBytes("yyyyyyyy");
257     for (int i = 0; i < MAX_PAGES; i++) {
258       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
259       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
260       assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
261       assertFalse(filterMPONE.filterRow());
262     }
263 
264     /* Now let's fill the page filter */
265     rowkey = Bytes.toBytes("xxxxxxx");
266     for (int i = 0; i < MAX_PAGES; i++) {
267       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
268       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
269       assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
270       assertFalse(filterMPONE.filterRow());
271     }
272 
273     /* We should still be able to include even though page filter is at max */
274     rowkey = Bytes.toBytes("yyy");
275     for (int i = 0; i < MAX_PAGES; i++) {
276       assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
277       KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i), Bytes.toBytes(i));
278       assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
279       assertFalse(filterMPONE.filterRow());
280     }
281   }
282 
283   /**
284    * When we do a "MUST_PASS_ONE" (a logical 'OR') of the above two filters
285    * we expect to get the same result as the 'prefix' only result.
286    * @throws Exception
287    */
288   @Test
289   public void testFilterListTwoFiltersMustPassOne() throws Exception {
290     byte[] r1 = Bytes.toBytes("Row1");
291     byte[] r11 = Bytes.toBytes("Row11");
292     byte[] r2 = Bytes.toBytes("Row2");
293 
294     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
295     flist.addFilter(new PrefixFilter(r1));
296     flist.filterRowKey(r1, 0, r1.length);
297     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
298     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
299 
300     flist.reset();
301     flist.filterRowKey(r2, 0, r2.length);
302     assertEquals(ReturnCode.SKIP, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
303 
304     flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
305     flist.addFilter(new AlwaysNextColFilter());
306     flist.addFilter(new PrefixFilter(r1));
307     flist.filterRowKey(r1, 0, r1.length);
308     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
309     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
310 
311     flist.reset();
312     flist.filterRowKey(r2, 0, r2.length);
313     assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
314   }
315 
316   /**
317    * When we do a "MUST_PASS_ONE" (a logical 'OR') of the two filters
318    * we expect to get the same result as the inclusive stop result.
319    * @throws Exception
320    */
321   @Test
322   public void testFilterListWithInclusiveStopFilterMustPassOne() throws Exception {
323     byte[] r1 = Bytes.toBytes("Row1");
324     byte[] r11 = Bytes.toBytes("Row11");
325     byte[] r2 = Bytes.toBytes("Row2");
326 
327     FilterList flist = new FilterList(FilterList.Operator.MUST_PASS_ONE);
328     flist.addFilter(new AlwaysNextColFilter());
329     flist.addFilter(new InclusiveStopFilter(r1));
330     flist.filterRowKey(r1, 0, r1.length);
331     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r1, r1, r1)));
332     assertEquals(ReturnCode.INCLUDE, flist.filterKeyValue(new KeyValue(r11, r11, r11)));
333 
334     flist.reset();
335     flist.filterRowKey(r2, 0, r2.length);
336     assertEquals(ReturnCode.NEXT_COL, flist.filterKeyValue(new KeyValue(r2, r2, r2)));
337   }
338 
339   static class AlwaysNextColFilter extends FilterBase {
340     public AlwaysNextColFilter() {
341       super();
342     }
343 
344     @Override
345     public ReturnCode filterKeyValue(Cell v) {
346       return ReturnCode.NEXT_COL;
347     }
348 
349     public static AlwaysNextColFilter parseFrom(final byte[] pbBytes)
350         throws DeserializationException {
351       return new AlwaysNextColFilter();
352     }
353   }
354 
355   /**
356    * Test serialization
357    * @throws Exception
358    */
359   @Test
360   public void testSerialization() throws Exception {
361     List<Filter> filters = new ArrayList<Filter>();
362     filters.add(new PageFilter(MAX_PAGES));
363     filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
364     Filter filterMPALL = new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
365 
366     // Decompose filterMPALL to bytes.
367     byte[] buffer = filterMPALL.toByteArray();
368 
369     // Recompose filterMPALL.
370     FilterList.parseFrom(buffer);
371 
372     // Run tests
373     mpOneTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getFilterMPONE())));
374     mpAllTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getMPALLFilter())));
375     orderingTest(ProtobufUtil.toFilter(ProtobufUtil.toFilter(getOrderingFilter())));
376   }
377 
378   /**
379    * Test filterKeyValue logic.
380    * @throws Exception
381    */
382   @Test
383   public void testFilterKeyValue() throws Exception {
384     Filter includeFilter = new FilterBase() {
385       @Override
386       public Filter.ReturnCode filterKeyValue(Cell v) {
387         return Filter.ReturnCode.INCLUDE;
388       }
389     };
390 
391     Filter alternateFilter = new FilterBase() {
392       boolean returnInclude = true;
393 
394       @Override
395       public Filter.ReturnCode filterKeyValue(Cell v) {
396         Filter.ReturnCode returnCode =
397             returnInclude ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.SKIP;
398         returnInclude = !returnInclude;
399         return returnCode;
400       }
401     };
402 
403     Filter alternateIncludeFilter = new FilterBase() {
404       boolean returnIncludeOnly = false;
405 
406       @Override
407       public Filter.ReturnCode filterKeyValue(Cell v) {
408         Filter.ReturnCode returnCode =
409             returnIncludeOnly ? Filter.ReturnCode.INCLUDE : Filter.ReturnCode.INCLUDE_AND_NEXT_COL;
410         returnIncludeOnly = !returnIncludeOnly;
411         return returnCode;
412       }
413     };
414 
415     // Check must pass one filter.
416     FilterList mpOnefilterList = new FilterList(Operator.MUST_PASS_ONE,
417         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
418     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
419     assertEquals(ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
420     // INCLUDE, SKIP, INCLUDE.
421     assertEquals(Filter.ReturnCode.INCLUDE, mpOnefilterList.filterKeyValue(null));
422 
423     // Check must pass all filter.
424     FilterList mpAllfilterList = new FilterList(Operator.MUST_PASS_ALL,
425         Arrays.asList(new Filter[] { includeFilter, alternateIncludeFilter, alternateFilter }));
426     // INCLUDE, INCLUDE, INCLUDE_AND_NEXT_COL.
427     assertEquals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL, mpAllfilterList.filterKeyValue(null));
428     // INCLUDE, SKIP, INCLUDE.
429     assertEquals(Filter.ReturnCode.SKIP, mpAllfilterList.filterKeyValue(null));
430   }
431 
432   /**
433    * Test pass-thru of hints.
434    */
435   @Test
436   public void testHintPassThru() throws Exception {
437 
438     final KeyValue minKeyValue = new KeyValue(Bytes.toBytes(0L), null, null);
439     final KeyValue maxKeyValue = new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
440 
441     Filter filterNoHint = new FilterBase() {
442       @Override
443       public byte[] toByteArray() {
444         return null;
445       }
446 
447       @Override
448       public ReturnCode filterKeyValue(Cell ignored) throws IOException {
449         return ReturnCode.INCLUDE;
450       }
451     };
452 
453     Filter filterMinHint = new FilterBase() {
454       @Override
455       public ReturnCode filterKeyValue(Cell ignored) {
456         return ReturnCode.SEEK_NEXT_USING_HINT;
457       }
458 
459       @Override
460       public Cell getNextCellHint(Cell currentKV) {
461         return minKeyValue;
462       }
463 
464       @Override
465       public byte[] toByteArray() {
466         return null;
467       }
468     };
469 
470     Filter filterMaxHint = new FilterBase() {
471       @Override
472       public ReturnCode filterKeyValue(Cell ignored) {
473         return ReturnCode.SEEK_NEXT_USING_HINT;
474       }
475 
476       @Override
477       public Cell getNextCellHint(Cell cell) {
478         return new KeyValue(Bytes.toBytes(Long.MAX_VALUE), null, null);
479       }
480 
481       @Override
482       public byte[] toByteArray() {
483         return null;
484       }
485     };
486 
487     // MUST PASS ONE
488 
489     // Should take the min if given two hints
490     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE,
491         Arrays.asList(new Filter[] { filterMinHint, filterMaxHint }));
492     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
493 
494     // Should have no hint if any filter has no hint
495     filterList = new FilterList(Operator.MUST_PASS_ONE,
496         Arrays.asList(new Filter[] { filterMinHint, filterMaxHint, filterNoHint }));
497     assertNull(filterList.getNextCellHint(null));
498     filterList = new FilterList(Operator.MUST_PASS_ONE,
499         Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
500     assertNull(filterList.getNextCellHint(null));
501 
502     // Should give max hint if its the only one
503     filterList = new FilterList(Operator.MUST_PASS_ONE,
504         Arrays.asList(new Filter[] { filterMaxHint, filterMaxHint }));
505     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
506 
507     // MUST PASS ALL
508 
509     // Should take the first hint
510     filterList = new FilterList(Operator.MUST_PASS_ALL,
511         Arrays.asList(new Filter[] { filterMinHint, filterMaxHint }));
512     filterList.filterKeyValue(null);
513     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
514 
515     filterList = new FilterList(Operator.MUST_PASS_ALL,
516         Arrays.asList(new Filter[] { filterMaxHint, filterMinHint }));
517     filterList.filterKeyValue(null);
518     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
519 
520     // Should have first hint even if a filter has no hint
521     filterList = new FilterList(Operator.MUST_PASS_ALL,
522         Arrays.asList(new Filter[] { filterNoHint, filterMinHint, filterMaxHint }));
523     filterList.filterKeyValue(null);
524     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
525     filterList = new FilterList(Operator.MUST_PASS_ALL,
526         Arrays.asList(new Filter[] { filterNoHint, filterMaxHint }));
527     filterList.filterKeyValue(null);
528     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), maxKeyValue));
529     filterList = new FilterList(Operator.MUST_PASS_ALL,
530         Arrays.asList(new Filter[] { filterNoHint, filterMinHint }));
531     filterList.filterKeyValue(null);
532     assertEquals(0, KeyValue.COMPARATOR.compare(filterList.getNextCellHint(null), minKeyValue));
533   }
534 
535   /**
536    * Tests the behavior of transform() in a hierarchical filter. transform() only applies after a
537    * filterKeyValue() whose return-code includes the KeyValue. Lazy evaluation of AND
538    */
539   @Test
540   public void testTransformMPO() throws Exception {
541     // Apply the following filter:
542     // (family=fam AND qualifier=qual1 AND KeyOnlyFilter)
543     // OR (family=fam AND qualifier=qual2)
544     final FilterList flist =
545         new FilterList(Operator.MUST_PASS_ONE,
546             Lists.<Filter> newArrayList(
547               new FilterList(Operator.MUST_PASS_ALL,
548                   Lists.<Filter> newArrayList(
549                     new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
550                     new QualifierFilter(CompareOp.EQUAL,
551                         new BinaryComparator(Bytes.toBytes("qual1"))),
552                     new KeyOnlyFilter())),
553               new FilterList(Operator.MUST_PASS_ALL,
554                   Lists.<Filter> newArrayList(
555                     new FamilyFilter(CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes("fam"))),
556                     new QualifierFilter(CompareOp.EQUAL,
557                         new BinaryComparator(Bytes.toBytes("qual2")))))));
558 
559     final KeyValue kvQual1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
560         Bytes.toBytes("qual1"), Bytes.toBytes("value"));
561     final KeyValue kvQual2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
562         Bytes.toBytes("qual2"), Bytes.toBytes("value"));
563     final KeyValue kvQual3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
564         Bytes.toBytes("qual3"), Bytes.toBytes("value"));
565 
566     // Value for fam:qual1 should be stripped:
567     assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual1));
568     final KeyValue transformedQual1 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual1));
569     assertEquals(0, transformedQual1.getValueLength());
570 
571     // Value for fam:qual2 should not be stripped:
572     assertEquals(Filter.ReturnCode.INCLUDE, flist.filterKeyValue(kvQual2));
573     final KeyValue transformedQual2 = KeyValueUtil.ensureKeyValue(flist.transformCell(kvQual2));
574     assertEquals("value", Bytes.toString(transformedQual2.getValueArray(),
575       transformedQual2.getValueOffset(), transformedQual2.getValueLength()));
576 
577     // Other keys should be skipped:
578     assertEquals(Filter.ReturnCode.SKIP, flist.filterKeyValue(kvQual3));
579   }
580 
581   @Test
582   public void testWithMultiVersionsInSameRow() throws Exception {
583     FilterList filterList01 =
584         new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 0));
585 
586     KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
587         1, Bytes.toBytes("value"));
588     KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
589         2, Bytes.toBytes("value"));
590     KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("qual"),
591         3, Bytes.toBytes("value"));
592 
593     assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList01.filterKeyValue(kv1));
594     assertEquals(ReturnCode.NEXT_COL, filterList01.filterKeyValue(kv2));
595     assertEquals(ReturnCode.NEXT_COL, filterList01.filterKeyValue(kv3));
596 
597     FilterList filterList11 =
598         new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, 1));
599 
600     assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv1));
601     assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv2));
602     assertEquals(ReturnCode.NEXT_COL, filterList11.filterKeyValue(kv3));
603   }
604 
605   @Test
606   public void testMPONEWithSeekNextUsingHint() throws Exception {
607     byte[] col = Bytes.toBytes("c");
608     FilterList filterList =
609         new FilterList(Operator.MUST_PASS_ONE, new ColumnPaginationFilter(1, col));
610 
611     KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 1,
612         Bytes.toBytes("value"));
613     KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 2,
614         Bytes.toBytes("value"));
615     KeyValue kv3 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 3,
616         Bytes.toBytes("value"));
617     KeyValue kv4 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"), Bytes.toBytes("c"), 4,
618         Bytes.toBytes("value"));
619 
620     assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
621     assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv2));
622     assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3));
623     assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(kv4));
624   }
625 
626   private static class MockFilter extends FilterBase {
627     private ReturnCode targetRetCode;
628     public boolean didCellPassToTheFilter = false;
629 
630     public MockFilter(ReturnCode targetRetCode) {
631       this.targetRetCode = targetRetCode;
632     }
633 
634     @Override
635     public ReturnCode filterKeyValue(Cell v) throws IOException {
636       this.didCellPassToTheFilter = true;
637       return targetRetCode;
638     }
639 
640     @Override
641     public boolean equals(Object obj) {
642       if(obj == null || !(obj instanceof  MockFilter)){
643         return false;
644       }
645       if(obj == this){
646         return true;
647       }
648       MockFilter f = (MockFilter)obj;
649       return this.targetRetCode.equals(f.targetRetCode);
650     }
651 
652     @Override
653     public int hashCode() {
654       return Objects.hash(this.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.filterKeyValue(kv1);
673     assertTrue(mockFilter.didCellPassToTheFilter);
674 
675     mockFilter.didCellPassToTheFilter = false;
676     filter.filterKeyValue(kv2);
677     assertFalse(mockFilter.didCellPassToTheFilter);
678 
679     mockFilter.didCellPassToTheFilter = false;
680     filter.filterKeyValue(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.filterKeyValue(kv1);
687     assertTrue(mockFilter.didCellPassToTheFilter);
688 
689     mockFilter.didCellPassToTheFilter = false;
690     filter.filterKeyValue(kv2);
691     assertFalse(mockFilter.didCellPassToTheFilter);
692 
693     mockFilter.didCellPassToTheFilter = false;
694     filter.filterKeyValue(kv3);
695     assertTrue(mockFilter.didCellPassToTheFilter);
696 
697     mockFilter = new MockFilter(ReturnCode.NEXT_ROW);
698     filter = new FilterList(Operator.MUST_PASS_ONE, mockFilter);
699     filter.filterKeyValue(kv1);
700     assertTrue(mockFilter.didCellPassToTheFilter);
701 
702     mockFilter.didCellPassToTheFilter = false;
703     filter.filterKeyValue(kv2);
704     assertFalse(mockFilter.didCellPassToTheFilter);
705 
706     mockFilter.didCellPassToTheFilter = false;
707     filter.filterKeyValue(kv3);
708     assertFalse(mockFilter.didCellPassToTheFilter);
709 
710     filter.reset();
711     mockFilter.didCellPassToTheFilter = false;
712     filter.filterKeyValue(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.filterKeyValue(kv1);
718     assertTrue(mockFilter.didCellPassToTheFilter);
719 
720     mockFilter.didCellPassToTheFilter = false;
721     filter.filterKeyValue(kv2);
722     assertFalse(mockFilter.didCellPassToTheFilter);
723 
724     mockFilter.didCellPassToTheFilter = false;
725     filter.filterKeyValue(kv3);
726     assertFalse(mockFilter.didCellPassToTheFilter);
727 
728     filter.reset();
729     mockFilter.didCellPassToTheFilter = false;
730     filter.filterKeyValue(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 MockFilter(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.filterKeyValue(kv1));
748 
749     filterList = new FilterList(Operator.MUST_PASS_ALL, filter2, filter3);
750     assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
751 
752     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter5, filter6);
753     assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(kv1));
754 
755     filterList = new FilterList(Operator.MUST_PASS_ALL, filter4, filter6);
756     assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(kv1));
757 
758     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter1);
759     assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv1));
760 
761     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter2, filter1, filter5);
762     assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(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.filterKeyValue(kv1));
767 
768     filterList = new FilterList(Operator.MUST_PASS_ALL, filter3, filter7);
769     assertEquals(ReturnCode.NEXT_ROW, filterList.filterKeyValue(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 MockFilter(ReturnCode.SEEK_NEXT_USING_HINT);
782     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2);
783     assertEquals(filterList.filterKeyValue(kv1), ReturnCode.INCLUDE);
784 
785     filterList = new FilterList(Operator.MUST_PASS_ONE, filter2, filter3);
786     assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv1));
787 
788     filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter5, filter6);
789     assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
790 
791     filterList = new FilterList(Operator.MUST_PASS_ONE, filter4, filter6);
792     assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(kv1));
793 
794     filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter1);
795     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
796 
797     filterList = new FilterList(Operator.MUST_PASS_ONE, filter3, filter2, filter1, filter5);
798     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(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.filterKeyValue(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.filterKeyValue(kv1));
807 
808     filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
809     assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(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 filterKeyValue(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.filterKeyValue(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.filterKeyValue(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(r22, 0, r22.length);
893     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
894     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
895 
896     filterList.reset();
897     filterList.filterRowKey(r1, 0, r1.length);
898     assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(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(r22, 0, r22.length);
909     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r22, r22, r22)));
910     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new KeyValue(r2, r2, r2)));
911 
912     filterList.reset();
913     filterList.filterRowKey(r1, 0, r1.length);
914     assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(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         CompareOp.EQUAL, Bytes.toBytes("value1"));
927     Filter filter2 = new SingleColumnValueFilter(Bytes.toBytes("cf"), Bytes.toBytes("column1"),
928         CompareOp.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.filterKeyValue(kv1));
935     c = keyOnlyFilterFirst.transformCell(kv1);
936     assertEquals(0, c.getValueLength());
937     assertEquals(ReturnCode.INCLUDE, keyOnlyFilterFirst.filterKeyValue(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.filterKeyValue(kv1));
945     c = keyOnlyFilterLast.transformCell(kv1);
946     assertEquals(0, c.getValueLength());
947     assertEquals(ReturnCode.INCLUDE, keyOnlyFilterLast.filterKeyValue(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.filterKeyValue(kv));
958     assertEquals(kv, filterList.transformCell(kv));
959 
960     filterList = new FilterList(Operator.MUST_PASS_ONE);
961     assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
962     assertEquals(kv, filterList.transformCell(kv));
963   }
964 
965   private static class MockNextRowFilter extends FilterBase {
966     private int hitCount = 0;
967 
968     public ReturnCode filterKeyValue(Cell v) throws IOException {
969       hitCount++;
970       return ReturnCode.NEXT_ROW;
971     }
972 
973     public int getHitCount() {
974       return hitCount;
975     }
976   }
977 
978   @Test
979   public void testRowCountFilter() throws IOException {
980     KeyValue kv1 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam1"), Bytes.toBytes("a"), 1,
981         Bytes.toBytes("value"));
982     KeyValue kv2 = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam2"), Bytes.toBytes("a"), 2,
983         Bytes.toBytes("value"));
984     MockNextRowFilter mockNextRowFilter = new MockNextRowFilter();
985     FilterList filter = new FilterList(Operator.MUST_PASS_ONE, mockNextRowFilter);
986     filter.filterKeyValue(kv1);
987     filter.filterKeyValue(kv2);
988     assertEquals(2, mockNextRowFilter.getHitCount());
989   }
990 
991   private static class TransformFilter extends FilterBase {
992     private ReturnCode targetRetCode;
993     private boolean transformed = false;
994 
995     public TransformFilter(ReturnCode targetRetCode) {
996       this.targetRetCode = targetRetCode;
997     }
998 
999     @Override
1000     public ReturnCode filterKeyValue(final Cell v) throws IOException {
1001       return targetRetCode;
1002     }
1003 
1004     @Override
1005     public Cell transformCell(Cell c) throws IOException {
1006       transformed = true;
1007       return super.transformCell(c);
1008     }
1009 
1010     public boolean getTransformed() {
1011       return this.transformed;
1012     }
1013 
1014     @Override
1015     public boolean equals(Object obj) {
1016       if(!(obj instanceof  TransformFilter)){
1017         return false;
1018       }
1019       if (obj == this) {
1020         return true;
1021       }
1022       TransformFilter f = (TransformFilter)obj;
1023       return this.targetRetCode.equals(f.targetRetCode);
1024     }
1025 
1026     @Override
1027     public int hashCode() {
1028       return Objects.hash(this.targetRetCode);
1029     }
1030   }
1031 
1032   @Test
1033   public void testTransformCell() throws IOException {
1034     KeyValue kv = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("cf"), Bytes.toBytes("column1"),
1035         1, Bytes.toBytes("value"));
1036 
1037     // case MUST_PASS_ONE
1038     TransformFilter filter1 = new TransformFilter(ReturnCode.INCLUDE);
1039     TransformFilter filter2 = new TransformFilter(ReturnCode.NEXT_ROW);
1040     TransformFilter filter3 = new TransformFilter(ReturnCode.SEEK_NEXT_USING_HINT);
1041     FilterList filterList = new FilterList(Operator.MUST_PASS_ONE, filter1, filter2, filter3);
1042     Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv));
1043     Assert.assertEquals(kv, filterList.transformCell(kv));
1044     Assert.assertEquals(true, filter1.getTransformed());
1045     Assert.assertEquals(false, filter2.getTransformed());
1046     Assert.assertEquals(false, filter3.getTransformed());
1047 
1048     // case MUST_PASS_ALL
1049     filter1 = new TransformFilter(ReturnCode.INCLUDE);
1050     filter2 = new TransformFilter(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
1051     filter3 = new TransformFilter(ReturnCode.INCLUDE_AND_NEXT_COL);
1052     filterList = new FilterList(Operator.MUST_PASS_ALL, filter1, filter2, filter3);
1053 
1054     Assert.assertEquals(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, filterList.filterKeyValue(kv));
1055     Assert.assertEquals(kv, filterList.transformCell(kv));
1056     Assert.assertEquals(true, filter1.getTransformed());
1057     Assert.assertEquals(true, filter2.getTransformed());
1058     Assert.assertEquals(true, filter3.getTransformed());
1059   }
1060 
1061   @Test
1062   public void testFilterListWithORWhenPassingCellMismatchPreviousRC() throws IOException {
1063     // Mainly test FilterListWithOR#calculateReturnCodeByPrevCellAndRC method with two sub-filters.
1064     KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"),
1065         100, Bytes.toBytes("value"));
1066     KeyValue kv2 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("a"), 99,
1067         Bytes.toBytes("value"));
1068     KeyValue kv3 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), Bytes.toBytes("b"), 1,
1069         Bytes.toBytes("value"));
1070     KeyValue kv4 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fan"), Bytes.toBytes("a"), 1,
1071         Bytes.toBytes("value"));
1072     Filter subFilter1 = Mockito.mock(FilterBase.class);
1073     Mockito.when(subFilter1.filterKeyValue(kv1)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1074     Mockito.when(subFilter1.filterKeyValue(kv2)).thenReturn(ReturnCode.NEXT_COL);
1075     Mockito.when(subFilter1.filterKeyValue(kv3)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1076     Mockito.when(subFilter1.filterKeyValue(kv4)).thenReturn(ReturnCode.INCLUDE_AND_NEXT_COL);
1077 
1078     Filter subFilter2 = Mockito.mock(FilterBase.class);
1079     Mockito.when(subFilter2.filterKeyValue(kv1)).thenReturn(ReturnCode.SKIP);
1080     Mockito.when(subFilter2.filterKeyValue(kv2)).thenReturn(ReturnCode.NEXT_ROW);
1081     Mockito.when(subFilter2.filterKeyValue(kv3)).thenReturn(ReturnCode.NEXT_ROW);
1082     Mockito.when(subFilter2.filterKeyValue(kv4)).thenReturn(ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW);
1083 
1084     Filter filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1085     Assert.assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(kv1));
1086     Assert.assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(kv2));
1087     Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv3));
1088     Assert.assertEquals(ReturnCode.INCLUDE_AND_NEXT_COL, filterList.filterKeyValue(kv4));
1089 
1090     // One sub-filter will filterAllRemaining but other sub-filter will return SEEK_HINT
1091     subFilter1 = Mockito.mock(FilterBase.class);
1092     Mockito.when(subFilter1.filterAllRemaining()).thenReturn(true);
1093     Mockito.when(subFilter1.filterKeyValue(kv1)).thenReturn(ReturnCode.NEXT_ROW);
1094 
1095     subFilter2 = Mockito.mock(FilterBase.class);
1096     Mockito.when(subFilter2.filterKeyValue(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1097     filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1098     Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
1099 
1100     // Two sub-filter returns SEEK_NEXT_USING_HINT, then we should return SEEK_NEXT_USING_HINT.
1101     subFilter1 = Mockito.mock(FilterBase.class);
1102     Mockito.when(subFilter1.filterKeyValue(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1103 
1104     subFilter2 = Mockito.mock(FilterBase.class);
1105     Mockito.when(subFilter2.filterKeyValue(kv1)).thenReturn(ReturnCode.SEEK_NEXT_USING_HINT);
1106     filterList = new FilterList(Operator.MUST_PASS_ONE, subFilter1, subFilter2);
1107     Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, filterList.filterKeyValue(kv1));
1108   }
1109 }