HBASE-18411 Dividing FiterList into two separate sub-classes: FilterListWithOR 
, FilterListWithAND

Signed-off-by: zhangduo <zhang...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b32bff02
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b32bff02
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b32bff02

Branch: refs/heads/HBASE-18410
Commit: b32bff028d00fedee5fb6e1ae8c587fd9e5f3b1e
Parents: 9dd2dda
Author: huzheng <open...@gmail.com>
Authored: Tue Oct 10 20:01:48 2017 +0800
Committer: zhangduo <zhang...@apache.org>
Committed: Tue Oct 24 11:39:31 2017 +0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/filter/FilterList.java  | 661 ++-----------------
 .../hadoop/hbase/filter/FilterListBase.java     | 159 +++++
 .../hadoop/hbase/filter/FilterListWithAND.java  | 273 ++++++++
 .../hadoop/hbase/filter/FilterListWithOR.java   | 383 +++++++++++
 .../hadoop/hbase/filter/TestFilterList.java     |  89 +++
 5 files changed, 962 insertions(+), 603 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
index b518645..97392d1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -37,86 +38,60 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos;
 
 /**
- * Implementation of {@link Filter} that represents an ordered List of Filters
- * which will be evaluated with a specified boolean operator {@link 
Operator#MUST_PASS_ALL}
- * (<code>AND</code>) or {@link Operator#MUST_PASS_ONE} (<code>OR</code>).
- * Since you can use Filter Lists as children of Filter Lists, you can create a
- * hierarchy of filters to be evaluated.
- *
- * <br>
- * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon 
as one filter does
- * not include the KeyValue.
- *
- * <br>
- * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always 
evaluated.
- *
- * <br>
+ * Implementation of {@link Filter} that represents an ordered List of Filters 
which will be
+ * evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL} 
(<code>AND</code>) or
+ * {@link Operator#MUST_PASS_ONE} (<code>OR</code>). Since you can use Filter 
Lists as children of
+ * Filter Lists, you can create a hierarchy of filters to be evaluated. <br>
+ * {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon 
as one filter does not
+ * include the KeyValue. <br>
+ * {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always 
evaluated. <br>
  * Defaults to {@link Operator#MUST_PASS_ALL}.
  */
 @InterfaceAudience.Public
 final public class FilterList extends FilterBase {
+
   /** set operator */
   @InterfaceAudience.Public
-  public static enum Operator {
+  public enum Operator {
     /** !AND */
     MUST_PASS_ALL,
     /** !OR */
     MUST_PASS_ONE
   }
 
-  private static final int MAX_LOG_FILTERS = 5;
-  private Operator operator = Operator.MUST_PASS_ALL;
-  private final List<Filter> filters;
-  private Collection<Filter> seekHintFilters = new ArrayList<Filter>();
-
-  /**
-   * Save previous return code and previous cell for every filter in filter 
list. For MUST_PASS_ONE,
-   * we use the previous return code to decide whether we should pass current 
cell encountered to
-   * the filter. For MUST_PASS_ALL, the two list are meaningless.
-   */
-  private List<ReturnCode> prevFilterRCList = null;
-  private List<Cell> prevCellList = null;
-
-  /** Reference Cell used by {@link #transformCell(Cell)} for validation 
purpose. */
-  private Cell referenceCell = null;
-
-  /**
-   * When filtering a given Cell in {@link #filterKeyValue(Cell)},
-   * this stores the transformed Cell to be returned by {@link 
#transformCell(Cell)}.
-   *
-   * Individual filters transformation are applied only when the filter 
includes the Cell.
-   * Transformations are composed in the order specified by {@link #filters}.
-   */
-  private Cell transformedCell = null;
+  private Operator operator;
+  private FilterListBase filterListBase;
 
   /**
    * Constructor that takes a set of {@link Filter}s and an operator.
    * @param operator Operator to process filter set with.
-   * @param rowFilters Set of row filters.
+   * @param filters Set of row filters.
    */
-  public FilterList(final Operator operator, final List<Filter> rowFilters) {
-    reversed = checkAndGetReversed(rowFilters, reversed);
-    this.filters = new ArrayList<>(rowFilters);
+  public FilterList(final Operator operator, final List<Filter> filters) {
+    if (operator == Operator.MUST_PASS_ALL) {
+      filterListBase = new FilterListWithAND(filters);
+    } else if (operator == Operator.MUST_PASS_ONE) {
+      filterListBase = new FilterListWithOR(filters);
+    }
     this.operator = operator;
-    initPrevListForMustPassOne(rowFilters.size());
   }
 
   /**
    * Constructor that takes a set of {@link Filter}s. The default operator 
MUST_PASS_ALL is assumed.
    * All filters are cloned to internal list.
-   * @param rowFilters list of filters
+   * @param filters list of filters
    */
-  public FilterList(final List<Filter> rowFilters) {
-    this(Operator.MUST_PASS_ALL, rowFilters);
+  public FilterList(final List<Filter> filters) {
+    this(Operator.MUST_PASS_ALL, filters);
   }
 
   /**
    * Constructor that takes a var arg number of {@link Filter}s. The default 
operator MUST_PASS_ALL
    * is assumed.
-   * @param rowFilters
+   * @param filters
    */
-  public FilterList(final Filter... rowFilters) {
-    this(Operator.MUST_PASS_ALL, Arrays.asList(rowFilters));
+  public FilterList(final Filter... filters) {
+    this(Operator.MUST_PASS_ALL, Arrays.asList(filters));
   }
 
   /**
@@ -130,21 +105,10 @@ final public class FilterList extends FilterBase {
   /**
    * Constructor that takes a var arg number of {@link Filter}s and an 
operator.
    * @param operator Operator to process filter set with.
-   * @param rowFilters Filters to use
+   * @param filters Filters to use
    */
-  public FilterList(final Operator operator, final Filter... rowFilters) {
-    this(operator, Arrays.asList(rowFilters));
-  }
-
-  private void initPrevListForMustPassOne(int size) {
-    if (operator == Operator.MUST_PASS_ONE) {
-      if (this.prevFilterRCList == null) {
-        prevFilterRCList = new ArrayList<>(Collections.nCopies(size, null));
-      }
-      if (this.prevCellList == null) {
-        prevCellList = new ArrayList<>(Collections.nCopies(size, null));
-      }
-    }
+  public FilterList(final Operator operator, final Filter... filters) {
+    this(operator, Arrays.asList(filters));
   }
 
   /**
@@ -160,38 +124,15 @@ final public class FilterList extends FilterBase {
    * @return filters
    */
   public List<Filter> getFilters() {
-    return filters;
+    return filterListBase.getFilters();
   }
 
   public int size() {
-    return filters.size();
-  }
-
-  private boolean isEmpty() {
-    return filters.isEmpty();
-  }
-
-  private static boolean checkAndGetReversed(List<Filter> rowFilters, boolean 
defaultValue) {
-    if (rowFilters.isEmpty()) {
-      return defaultValue;
-    }
-    Boolean retValue = rowFilters.get(0).isReversed();
-    boolean allEqual = 
rowFilters.stream().map(Filter::isReversed).allMatch(retValue::equals);
-    if (!allEqual) {
-      throw new IllegalArgumentException("Filters in the list must have the 
same reversed flag");
-    }
-    return retValue;
+    return filterListBase.size();
   }
 
   public void addFilter(List<Filter> filters) {
-    if (checkAndGetReversed(filters, isReversed()) != isReversed()) {
-      throw new IllegalArgumentException("Filters in the list must have the 
same reversed flag");
-    }
-    this.filters.addAll(filters);
-    if (operator == Operator.MUST_PASS_ONE) {
-      this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null));
-      this.prevCellList.addAll(Collections.nCopies(filters.size(), null));
-    }
+    filterListBase.addFilterLists(filters);
   }
 
   /**
@@ -204,483 +145,51 @@ final public class FilterList extends FilterBase {
 
   @Override
   public void reset() throws IOException {
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      filters.get(i).reset();
-      if (operator == Operator.MUST_PASS_ONE) {
-        prevFilterRCList.set(i, null);
-        prevCellList.set(i, null);
-      }
-    }
-    seekHintFilters.clear();
+    filterListBase.reset();
   }
 
   @Override
   public boolean filterRowKey(byte[] rowKey, int offset, int length) throws 
IOException {
-    if (isEmpty()) {
-      return super.filterRowKey(rowKey, offset, length);
-    }
-    boolean flag = this.operator == Operator.MUST_PASS_ONE;
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      Filter filter = filters.get(i);
-      if (this.operator == Operator.MUST_PASS_ALL) {
-        if (filter.filterAllRemaining() || filter.filterRowKey(rowKey, offset, 
length)) {
-          flag = true;
-        }
-      } else if (this.operator == Operator.MUST_PASS_ONE) {
-        if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey, 
offset, length)) {
-          flag = false;
-        }
-      }
-    }
-    return flag;
+    return filterListBase.filterRowKey(rowKey, offset, length);
   }
 
   @Override
   public boolean filterRowKey(Cell firstRowCell) throws IOException {
-    if (isEmpty()) {
-      return super.filterRowKey(firstRowCell);
-    }
-    boolean flag = this.operator == Operator.MUST_PASS_ONE;
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      Filter filter = filters.get(i);
-      if (this.operator == Operator.MUST_PASS_ALL) {
-        if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) {
-          flag = true;
-        }
-      } else if (this.operator == Operator.MUST_PASS_ONE) {
-        if (!filter.filterAllRemaining() && 
!filter.filterRowKey(firstRowCell)) {
-          flag = false;
-        }
-      }
-    }
-    return flag;
+    return filterListBase.filterRowKey(firstRowCell);
   }
 
   @Override
   public boolean filterAllRemaining() throws IOException {
-    if (isEmpty()) {
-      return super.filterAllRemaining();
-    }
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      if (filters.get(i).filterAllRemaining()) {
-        if (operator == Operator.MUST_PASS_ALL) {
-          return true;
-        }
-      } else {
-        if (operator == Operator.MUST_PASS_ONE) {
-          return false;
-        }
-      }
-    }
-    return operator == Operator.MUST_PASS_ONE;
+    return filterListBase.filterAllRemaining();
   }
 
   @Override
   public Cell transformCell(Cell c) throws IOException {
-    if (isEmpty()) {
-      return super.transformCell(c);
-    }
-    if (!CellUtil.equals(c, referenceCell)) {
-      throw new IllegalStateException(
-          "Reference Cell: " + this.referenceCell + " does not match: " + c);
-    }
-    // Copy transformedCell into a new cell and reset transformedCell & 
referenceCell to null for
-    // Java GC optimization
-    Cell cell = KeyValueUtil.copyToNewKeyValue(this.transformedCell);
-    this.transformedCell = null;
-    this.referenceCell = null;
-    return cell;
-  }
-
-  /**
-   * For MUST_PASS_ONE, we cannot make sure that when filter-A in filter list 
return NEXT_COL then
-   * the next cell passing to filterList will be the first cell in next 
column, because if filter-B
-   * in filter list return SKIP, then the filter list will return SKIP. In 
this case, we should pass
-   * the cell following the previous cell, and it's possible that the next 
cell has the same column
-   * as the previous cell even if filter-A has NEXT_COL returned for the 
previous cell. So we should
-   * save the previous cell and the return code list when checking previous 
cell for every filter in
-   * filter list, and verify if currentCell fit the previous return code, if 
fit then pass the currentCell
-   * to the corresponding filter. (HBASE-17678)
-   */
-  private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell 
currentCell, int filterIdx)
-      throws IOException {
-    ReturnCode prevCode = this.prevFilterRCList.get(filterIdx);
-    if (prevCell == null || prevCode == null) {
-      return true;
-    }
-    switch (prevCode) {
-    case INCLUDE:
-    case SKIP:
-      return true;
-    case SEEK_NEXT_USING_HINT:
-      Cell nextHintCell = getNextCellHint(prevCell);
-      return nextHintCell == null
-          || CellComparatorImpl.COMPARATOR.compare(currentCell, nextHintCell) 
>= 0;
-    case NEXT_COL:
-    case INCLUDE_AND_NEXT_COL:
-      return !CellUtil.matchingRowColumn(prevCell, currentCell);
-    case NEXT_ROW:
-    case INCLUDE_AND_SEEK_NEXT_ROW:
-      return !CellUtil.matchingRows(prevCell, currentCell);
-    default:
-      throw new IllegalStateException("Received code is not valid.");
-    }
-  }
-
-  /**
-   * FilterList with MUST_PASS_ALL choose the maximal forward step among 
sub-filters in filter list.
-   * Let's call it: The Maximal Step Rule. So if filter-A in filter list 
return INCLUDE and filter-B
-   * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should 
return
-   * INCLUDE_AND_NEXT_COL. For SEEK_NEXT_USING_HINT, it's more special, and in 
method
-   * filterKeyValueWithMustPassAll(), if any sub-filter return 
SEEK_NEXT_USING_HINT, then our filter
-   * list will return SEEK_NEXT_USING_HINT. so we don't care about the 
SEEK_NEXT_USING_HINT here. <br/>
-   * <br/>
-   * The jump step will be:
-   *
-   * <pre>
-   * INCLUDE &lt; SKIP &lt; INCLUDE_AND_NEXT_COL &lt; NEXT_COL &lt; 
INCLUDE_AND_SEEK_NEXT_ROW &lt; NEXT_ROW &lt; SEEK_NEXT_USING_HINT
-   * </pre>
-   *
-   * Here, we have the following map to describe The Maximal Step Rule. if 
current return code (for
-   * previous sub-filters in filter list) is <strong>ReturnCode</strong>, and 
current filter returns
-   * <strong>localRC</strong>, then we should return map[ReturnCode][localRC] 
for the merged result,
-   * according to The Maximal Step Rule. <br/>
-   *
-   * <pre>
-   * LocalCode\ReturnCode       INCLUDE                    
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE                    INCLUDE                    
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE_AND_NEXT_COL       INCLUDE_AND_NEXT_COL       
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  NEXT_COL              
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE_AND_SEEK_NEXT_ROW  
INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW  NEXT_ROW              
NEXT_ROW              NEXT_ROW              SEEK_NEXT_USING_HINT
-   * SKIP                       SKIP                       NEXT_COL            
      NEXT_ROW                   SKIP                  NEXT_COL              
NEXT_ROW              SEEK_NEXT_USING_HINT
-   * NEXT_COL                   NEXT_COL                   NEXT_COL            
      NEXT_ROW                   NEXT_COL              NEXT_COL              
NEXT_ROW              SEEK_NEXT_USING_HINT
-   * NEXT_ROW                   NEXT_ROW                   NEXT_ROW            
      NEXT_ROW                   NEXT_ROW              NEXT_ROW              
NEXT_ROW              SEEK_NEXT_USING_HINT
-   * SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT       
SEEK_NEXT_USING_HINT      SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT  
SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT
-   * </pre>
-   * @param rc Return code which is calculated by previous sub-filter(s) in 
filter list.
-   * @param localRC Return code of the current sub-filter in filter list.
-   * @return Return code which is merged by the return code of previous 
sub-filter(s) and the return
-   *         code of current sub-filter.
-   */
-  private ReturnCode mergeReturnCodeForAndOperator(ReturnCode rc, ReturnCode 
localRC) {
-    if (rc == ReturnCode.SEEK_NEXT_USING_HINT || localRC == 
ReturnCode.SEEK_NEXT_USING_HINT) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    }
-    switch (localRC) {
-    case INCLUDE:
-      return rc;
-    case INCLUDE_AND_NEXT_COL:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL)) {
-        return ReturnCode.INCLUDE_AND_NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL)) {
-        return ReturnCode.NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) {
-        return ReturnCode.NEXT_ROW;
-      }
-      break;
-    case INCLUDE_AND_SEEK_NEXT_ROW:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW)) {
-        return ReturnCode.NEXT_ROW;
-      }
-      break;
-    case SKIP:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP)) {
-        return ReturnCode.SKIP;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.NEXT_COL)) {
-        return ReturnCode.NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
-        return ReturnCode.NEXT_ROW;
-      }
-      break;
-    case NEXT_COL:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.SKIP,
-        ReturnCode.NEXT_COL)) {
-        return ReturnCode.NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
-        return ReturnCode.NEXT_ROW;
-      }
-      break;
-    case NEXT_ROW:
-      return ReturnCode.NEXT_ROW;
-    }
-    throw new IllegalStateException("Received code is not valid. rc: " + rc + 
", localRC: "
-        + localRC);
-  }
-
-  private ReturnCode filterKeyValueWithMustPassAll(Cell c) throws IOException {
-    ReturnCode rc = ReturnCode.INCLUDE;
-    Cell transformed = c;
-    this.seekHintFilters.clear();
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      Filter filter = filters.get(i);
-      if (filter.filterAllRemaining()) {
-        return ReturnCode.NEXT_ROW;
-      }
-      ReturnCode localRC = filter.filterKeyValue(c);
-      rc = mergeReturnCodeForAndOperator(rc, localRC);
-
-      // For INCLUDE* case, we need to update the transformed cell.
-      if (isInReturnCodes(localRC, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        transformed = filter.transformCell(transformed);
-      }
-      if (localRC == ReturnCode.SEEK_NEXT_USING_HINT) {
-        seekHintFilters.add(filter);
-      }
-    }
-    this.transformedCell = transformed;
-    if (!seekHintFilters.isEmpty()) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    }
-    return rc;
-  }
-
-  private void updatePrevFilterRCList(int index, ReturnCode currentRC) {
-    prevFilterRCList.set(index, currentRC);
-  }
-
-  private void updatePrevCellList(int index, Cell currentCell, ReturnCode 
currentRC) {
-    if (currentCell == null || currentRC == ReturnCode.INCLUDE || currentRC == 
ReturnCode.SKIP) {
-      // If previous return code is INCLUDE or SKIP, we should always pass the 
next cell to the
-      // corresponding sub-filter(need not test 
shouldPassCurrentCellToFilter() method), So we
-      // need not save current cell to prevCellList for saving heap memory.
-      prevCellList.set(index, null);
-    } else {
-      prevCellList.set(index, KeyValueUtil.toNewKeyCell(currentCell));
-    }
-  }
-
-  private static boolean isInReturnCodes(ReturnCode testRC, ReturnCode... 
returnCodes) {
-    return Arrays.stream(returnCodes).anyMatch(testRC::equals);
-  }
-
-  /**
-   * FilterList with MUST_PASS_ONE choose the minimal forward step among 
sub-filter in filter list.
-   * Let's call it: The Minimal Step Rule. So if filter-A in filter list 
return INCLUDE and filter-B
-   * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should 
return INCLUDE. For
-   * SEEK_NEXT_USING_HINT, it's more special, because we do not know how far 
it will forward, so we
-   * use SKIP by default.<br/>
-   * <br/>
-   * The jump step will be:
-   *
-   * <pre>
-   * INCLUDE &lt; SKIP &lt; INCLUDE_AND_NEXT_COL &lt; NEXT_COL &lt; 
INCLUDE_AND_SEEK_NEXT_ROW &lt; NEXT_ROW &lt; SEEK_NEXT_USING_HINT
-   * </pre>
-   *
-   * Here, we have the following map to describe The Minimal Step Rule. if 
current return code (for
-   * previous sub-filters in filter list) is <strong>ReturnCode</strong>, and 
current filter returns
-   * <strong>localRC</strong>, then we should return map[ReturnCode][localRC] 
for the merged result,
-   * according to The Minimal Step Rule.<br/>
-   *
-   * <pre>
-   * LocalCode\ReturnCode       INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW             
     SEEK_NEXT_USING_HINT
-   * INCLUDE                    INCLUDE INCLUDE                  INCLUDE       
             INCLUDE   INCLUDE               INCLUDE                   INCLUDE
-   * INCLUDE_AND_NEXT_COL       INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_NEXT_COL       INCLUDE   INCLUDE_AND_NEXT_COL  INCLUDE_AND_NEXT_COL 
     INCLUDE
-   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE   INCLUDE_AND_NEXT_COL  
INCLUDE_AND_SEEK_NEXT_ROW INCLUDE
-   * SKIP                       INCLUDE INCLUDE                  INCLUDE       
             SKIP      SKIP                  SKIP                      SKIP
-   * NEXT_COL                   INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_NEXT_COL       SKIP      NEXT_COL              NEXT_COL             
     SKIP
-   * NEXT_ROW                   INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW             
     SKIP
-   * SEEK_NEXT_USING_HINT       INCLUDE INCLUDE                  INCLUDE       
             SKIP      SKIP                  SKIP                      
SEEK_NEXT_USING_HINT
-   * </pre>
-   * @param rc Return code which is calculated by previous sub-filter(s) in 
filter list.
-   * @param localRC Return code of the current sub-filter in filter list.
-   * @return Return code which is merged by the return code of previous 
sub-filter(s) and the return
-   *         code of current sub-filter.
-   */
-  private ReturnCode mergeReturnCodeForOrOperator(ReturnCode rc, ReturnCode 
localRC) {
-    if (rc == null) return localRC;
-    switch (localRC) {
-    case INCLUDE:
-      return ReturnCode.INCLUDE;
-    case INCLUDE_AND_NEXT_COL:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP,
-        ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW,
-        ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_NEXT_COL;
-      }
-      break;
-    case INCLUDE_AND_SEEK_NEXT_ROW:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP,
-        ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.NEXT_COL)) {
-        return ReturnCode.INCLUDE_AND_NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
-      }
-      break;
-    case SKIP:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW,
-        ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.SKIP;
-      }
-      break;
-    case NEXT_COL:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, 
ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.SKIP;
-      }
-      break;
-    case NEXT_ROW:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL)) {
-        return ReturnCode.INCLUDE_AND_NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, 
ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.SKIP;
-      }
-      if (isInReturnCodes(rc, ReturnCode.NEXT_COL)) {
-        return ReturnCode.NEXT_COL;
-      }
-      if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) {
-        return ReturnCode.NEXT_ROW;
-      }
-      break;
-    case SEEK_NEXT_USING_HINT:
-      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        return ReturnCode.INCLUDE;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW)) {
-        return ReturnCode.SKIP;
-      }
-      if (isInReturnCodes(rc, ReturnCode.SEEK_NEXT_USING_HINT)) {
-        return ReturnCode.SEEK_NEXT_USING_HINT;
-      }
-      break;
-    }
-    throw new IllegalStateException(
-        "Received code is not valid. rc: " + rc + ", localRC: " + localRC);
-  }
-
-  private ReturnCode filterKeyValueWithMustPassOne(Cell c) throws IOException {
-    ReturnCode rc = null;
-    boolean everyFilterReturnHint = true;
-    Cell transformed = c;
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      Filter filter = filters.get(i);
-      Cell prevCell = this.prevCellList.get(i);
-      if (filter.filterAllRemaining() || 
!shouldPassCurrentCellToFilter(prevCell, c, i)) {
-        everyFilterReturnHint = false;
-        continue;
-      }
-
-      ReturnCode localRC = filter.filterKeyValue(c);
-
-      // Update previous return code and previous cell for filter[i].
-      updatePrevFilterRCList(i, localRC);
-      updatePrevCellList(i, c, localRC);
-
-      if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
-        everyFilterReturnHint = false;
-      }
-
-      rc = mergeReturnCodeForOrOperator(rc, localRC);
-
-      // For INCLUDE* case, we need to update the transformed cell.
-      if (isInReturnCodes(localRC, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
-        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
-        transformed = filter.transformCell(transformed);
-      }
-    }
-
-    this.transformedCell = transformed;
-    if (everyFilterReturnHint) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    } else if (rc == null) {
-      // Each sub-filter in filter list got true for filterAllRemaining().
-      return ReturnCode.SKIP;
-    } else {
-      return rc;
-    }
+    return filterListBase.transformCell(c);
   }
 
   @Override
   public ReturnCode filterKeyValue(Cell c) throws IOException {
-    if (isEmpty()) {
-      return ReturnCode.INCLUDE;
-    }
-    this.referenceCell = c;
-
-    if (operator == Operator.MUST_PASS_ALL) {
-      return filterKeyValueWithMustPassAll(c);
-    } else {
-      return filterKeyValueWithMustPassOne(c);
-    }
+    return filterListBase.filterKeyValue(c);
   }
 
   /**
-   * Filters that never filter by modifying the returned List of Cells can
-   * inherit this implementation that does nothing.
-   *
-   * {@inheritDoc}
+   * Filters that never filter by modifying the returned List of Cells can 
inherit this
+   * implementation that does nothing. {@inheritDoc}
    */
   @Override
   public void filterRowCells(List<Cell> cells) throws IOException {
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      filters.get(i).filterRowCells(cells);
-    }
+    filterListBase.filterRowCells(cells);
   }
 
   @Override
   public boolean hasFilterRow() {
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      if (filters.get(i).hasFilterRow()) {
-        return true;
-      }
-    }
-    return false;
+    return filterListBase.hasFilterRow();
   }
 
   @Override
   public boolean filterRow() throws IOException {
-    if (isEmpty()) {
-      return super.filterRow();
-    }
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      Filter filter = filters.get(i);
-      if (operator == Operator.MUST_PASS_ALL) {
-        if (filter.filterRow()) {
-          return true;
-        }
-      } else if (operator == Operator.MUST_PASS_ONE) {
-        if (!filter.filterRow()) {
-          return false;
-        }
-      }
-    }
-    return operator == Operator.MUST_PASS_ONE;
+    return filterListBase.filterRow();
   }
 
   /**
@@ -689,6 +198,7 @@ final public class FilterList extends FilterBase {
   public byte[] toByteArray() throws IOException {
     FilterProtos.FilterList.Builder builder = 
FilterProtos.FilterList.newBuilder();
     
builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
+    ArrayList<Filter> filters = filterListBase.getFilters();
     for (int i = 0, n = filters.size(); i < n; i++) {
       builder.addFilters(ProtobufUtil.toFilter(filters.get(i)));
     }
@@ -701,8 +211,7 @@ final public class FilterList extends FilterBase {
    * @throws DeserializationException
    * @see #toByteArray
    */
-  public static FilterList parseFrom(final byte [] pbBytes)
-      throws DeserializationException {
+  public static FilterList parseFrom(final byte[] pbBytes) throws 
DeserializationException {
     FilterProtos.FilterList proto;
     try {
       proto = FilterProtos.FilterList.parseFrom(pbBytes);
@@ -724,96 +233,42 @@ final public class FilterList extends FilterBase {
 
   /**
    * @param other
-   * @return true if and only if the fields of the filter that are serialized
-   * are equal to the corresponding fields in other.  Used for testing.
+   * @return true if and only if the fields of the filter that are serialized 
are equal to the
+   *         corresponding fields in other. Used for testing.
    */
   boolean areSerializedFieldsEqual(Filter other) {
     if (other == this) return true;
     if (!(other instanceof FilterList)) return false;
 
-    FilterList o = (FilterList)other;
-    return this.getOperator().equals(o.getOperator()) &&
-      ((this.getFilters() == o.getFilters())
-      || this.getFilters().equals(o.getFilters()));
+    FilterList o = (FilterList) other;
+    return this.getOperator().equals(o.getOperator())
+        && ((this.getFilters() == o.getFilters()) || 
this.getFilters().equals(o.getFilters()));
   }
 
   @Override
   public Cell getNextCellHint(Cell currentCell) throws IOException {
-    if (isEmpty()) {
-      return super.getNextCellHint(currentCell);
-    }
-    Cell keyHint = null;
-    if (operator == Operator.MUST_PASS_ALL) {
-      for (Filter filter : seekHintFilters) {
-        if (filter.filterAllRemaining()) continue;
-        Cell curKeyHint = filter.getNextCellHint(currentCell);
-        if (keyHint == null) {
-          keyHint = curKeyHint;
-          continue;
-        }
-        if (CellComparatorImpl.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
-          keyHint = curKeyHint;
-        }
-      }
-      return keyHint;
-    }
-
-    // If any condition can pass, we need to keep the min hint
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      if (filters.get(i).filterAllRemaining()) {
-        continue;
-      }
-      Cell curKeyHint = filters.get(i).getNextCellHint(currentCell);
-      if (curKeyHint == null) {
-        // If we ever don't have a hint and this is must-pass-one, then no hint
-        return null;
-      }
-      // If this is the first hint we find, set it
-      if (keyHint == null) {
-        keyHint = curKeyHint;
-        continue;
-      }
-      if (CellComparatorImpl.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
-        keyHint = curKeyHint;
-      }
-    }
-    return keyHint;
+    return this.filterListBase.getNextCellHint(currentCell);
   }
 
   @Override
   public boolean isFamilyEssential(byte[] name) throws IOException {
-    if (isEmpty()) {
-      return super.isFamilyEssential(name);
-    }
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      if (filters.get(i).isFamilyEssential(name)) {
-        return true;
-      }
-    }
-    return false;
+    return this.filterListBase.isFamilyEssential(name);
   }
 
   @Override
   public void setReversed(boolean reversed) {
-    for (int i = 0, n = filters.size(); i < n; i++) {
-      filters.get(i).setReversed(reversed);
-    }
     this.reversed = reversed;
+    this.filterListBase.setReversed(reversed);
   }
 
   @Override
-  public String toString() {
-    return toString(MAX_LOG_FILTERS);
+  public boolean isReversed() {
+    assert this.reversed == this.filterListBase.isReversed();
+    return this.reversed;
   }
 
-  protected String toString(int maxFilters) {
-    int endIndex = this.filters.size() < maxFilters
-        ? this.filters.size() : maxFilters;
-    return String.format("%s %s (%d/%d): %s",
-        this.getClass().getSimpleName(),
-        this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
-        endIndex,
-        this.filters.size(),
-        this.filters.subList(0, endIndex).toString());
+  @Override
+  public String toString() {
+    return this.filterListBase.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java
new file mode 100644
index 0000000..7fa0245
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java
@@ -0,0 +1,159 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparatorImpl;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Base class for FilterList. Currently, we have two sub-classes to extend 
this class:
+ * {@link FilterListWithOR}, {@link FilterListWithAND}.
+ */
+@InterfaceAudience.Private
+public abstract class FilterListBase extends FilterBase {
+  private static final int MAX_LOG_FILTERS = 5;
+  protected final ArrayList<Filter> filters;
+
+  /** Reference Cell used by {@link #transformCell(Cell)} for validation 
purpose. */
+  protected Cell referenceCell = null;
+
+  /**
+   * When filtering a given Cell in {@link #filterKeyValue(Cell)}, this stores 
the transformed Cell
+   * to be returned by {@link #transformCell(Cell)}. Individual filters 
transformation are applied
+   * only when the filter includes the Cell. Transformations are composed in 
the order specified by
+   * {@link #filters}.
+   */
+  protected Cell transformedCell = null;
+
+  public FilterListBase(List<Filter> filters) {
+    reversed = checkAndGetReversed(filters, reversed);
+    this.filters = new ArrayList<>(filters);
+  }
+
+  protected static boolean isInReturnCodes(ReturnCode testRC, ReturnCode... 
returnCodes) {
+    return Arrays.stream(returnCodes).anyMatch(testRC::equals);
+  }
+
+  protected static boolean checkAndGetReversed(List<Filter> rowFilters, 
boolean defaultValue) {
+    if (rowFilters.isEmpty()) {
+      return defaultValue;
+    }
+    Boolean retValue = rowFilters.get(0).isReversed();
+    boolean allEqual = 
rowFilters.stream().map(Filter::isReversed).allMatch(retValue::equals);
+    if (!allEqual) {
+      throw new IllegalArgumentException("Filters in the list must have the 
same reversed flag");
+    }
+    return retValue;
+  }
+
+  public abstract void addFilterLists(List<Filter> filters);
+
+  public int size() {
+    return this.filters.size();
+  }
+
+  public boolean isEmpty() {
+    return this.filters.isEmpty();
+  }
+
+  public ArrayList<Filter> getFilters() {
+    return this.filters;
+  }
+
+  protected int compareCell(Cell a, Cell b) {
+    int cmp = CellComparatorImpl.COMPARATOR.compare(a, b);
+    return reversed ? -1 * cmp : cmp;
+  }
+
+  @Override
+  public Cell transformCell(Cell c) throws IOException {
+    if (isEmpty()) {
+      return super.transformCell(c);
+    }
+    if (!CellUtil.equals(c, referenceCell)) {
+      throw new IllegalStateException(
+          "Reference Cell: " + this.referenceCell + " does not match: " + c);
+    }
+    // Copy transformedCell into a new cell and reset transformedCell & 
referenceCell to null for
+    // Java GC optimization
+    Cell cell = KeyValueUtil.copyToNewKeyValue(this.transformedCell);
+    this.transformedCell = null;
+    this.referenceCell = null;
+    return cell;
+  }
+
+  /**
+   * Filters that never filter by modifying the returned List of Cells can 
inherit this
+   * implementation that does nothing. {@inheritDoc}
+   */
+  @Override
+  public void filterRowCells(List<Cell> cells) throws IOException {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      filters.get(i).filterRowCells(cells);
+    }
+  }
+
+  @Override
+  public boolean hasFilterRow() {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      if (filters.get(i).hasFilterRow()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public boolean isFamilyEssential(byte[] name) throws IOException {
+    if (this.filters.isEmpty()) {
+      return super.isFamilyEssential(name);
+    }
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      if (filters.get(i).isFamilyEssential(name)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public void setReversed(boolean reversed) {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      filters.get(i).setReversed(reversed);
+    }
+    this.reversed = reversed;
+  }
+
+  @Override
+  public String toString() {
+    int endIndex = this.size() < MAX_LOG_FILTERS ? this.size() : 
MAX_LOG_FILTERS;
+    return formatLogFilters(filters.subList(0, endIndex));
+  }
+
+  protected abstract String formatLogFilters(List<Filter> logFilters);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
new file mode 100644
index 0000000..fa979c0
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
@@ -0,0 +1,273 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * FilterListWithAND represents an ordered list of filters which will be 
evaluated with an AND
+ * operator.
+ */
+@InterfaceAudience.Private
+public class FilterListWithAND extends FilterListBase {
+
+  private Set<Filter> seekHintFilter = new HashSet<>();
+
+  public FilterListWithAND(List<Filter> filters) {
+    super(filters);
+  }
+
+  @Override
+  public void addFilterLists(List<Filter> filters) {
+    if (checkAndGetReversed(filters, isReversed()) != isReversed()) {
+      throw new IllegalArgumentException("Filters in the list must have the 
same reversed flag");
+    }
+    this.filters.addAll(filters);
+  }
+
+  @Override
+  protected String formatLogFilters(List<Filter> logFilters) {
+    return String.format("FilterList AND (%d/%d): %s", logFilters.size(), 
this.size(),
+      logFilters.toString());
+  }
+
+  /**
+   * FilterList with MUST_PASS_ALL choose the maximal forward step among 
sub-filters in filter list.
+   * Let's call it: The Maximal Step Rule. So if filter-A in filter list 
return INCLUDE and filter-B
+   * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should 
return
+   * INCLUDE_AND_NEXT_COL. For SEEK_NEXT_USING_HINT, it's more special, and in 
method
+   * filterKeyValueWithMustPassAll(), if any sub-filter return 
SEEK_NEXT_USING_HINT, then our filter
+   * list will return SEEK_NEXT_USING_HINT. so we don't care about the 
SEEK_NEXT_USING_HINT here.
+   * <br/>
+   * <br/>
+   * The jump step will be:
+   *
+   * <pre>
+   * INCLUDE &lt; SKIP &lt; INCLUDE_AND_NEXT_COL &lt; NEXT_COL &lt; 
INCLUDE_AND_SEEK_NEXT_ROW &lt; NEXT_ROW &lt; SEEK_NEXT_USING_HINT
+   * </pre>
+   *
+   * Here, we have the following map to describe The Maximal Step Rule. if 
current return code (for
+   * previous sub-filters in filter list) is <strong>ReturnCode</strong>, and 
current filter returns
+   * <strong>localRC</strong>, then we should return map[ReturnCode][localRC] 
for the merged result,
+   * according to The Maximal Step Rule. <br/>
+   *
+   * <pre>
+   * LocalCode\ReturnCode       INCLUDE                    
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
+   * INCLUDE                    INCLUDE                    
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  SKIP                  
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
+   * INCLUDE_AND_NEXT_COL       INCLUDE_AND_NEXT_COL       
INCLUDE_AND_NEXT_COL      INCLUDE_AND_SEEK_NEXT_ROW  NEXT_COL              
NEXT_COL              NEXT_ROW              SEEK_NEXT_USING_HINT
+   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE_AND_SEEK_NEXT_ROW  
INCLUDE_AND_SEEK_NEXT_ROW INCLUDE_AND_SEEK_NEXT_ROW  NEXT_ROW              
NEXT_ROW              NEXT_ROW              SEEK_NEXT_USING_HINT
+   * SKIP                       SKIP                       NEXT_COL            
      NEXT_ROW                   SKIP                  NEXT_COL              
NEXT_ROW              SEEK_NEXT_USING_HINT
+   * NEXT_COL                   NEXT_COL                   NEXT_COL            
      NEXT_ROW                   NEXT_COL              NEXT_COL              
NEXT_ROW              SEEK_NEXT_USING_HINT
+   * NEXT_ROW                   NEXT_ROW                   NEXT_ROW            
      NEXT_ROW                   NEXT_ROW              NEXT_ROW              
NEXT_ROW              SEEK_NEXT_USING_HINT
+   * SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT       
SEEK_NEXT_USING_HINT      SEEK_NEXT_USING_HINT       SEEK_NEXT_USING_HINT  
SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT  SEEK_NEXT_USING_HINT
+   * </pre>
+   *
+   * @param rc Return code which is calculated by previous sub-filter(s) in 
filter list.
+   * @param localRC Return code of the current sub-filter in filter list.
+   * @return Return code which is merged by the return code of previous 
sub-filter(s) and the return
+   *         code of current sub-filter.
+   */
+  private ReturnCode mergeReturnCode(ReturnCode rc, ReturnCode localRC) {
+    if (rc == ReturnCode.SEEK_NEXT_USING_HINT || localRC == 
ReturnCode.SEEK_NEXT_USING_HINT) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+    switch (localRC) {
+    case INCLUDE:
+      return rc;
+    case INCLUDE_AND_NEXT_COL:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL)) {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL)) {
+        return ReturnCode.NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_ROW;
+      }
+      break;
+    case INCLUDE_AND_SEEK_NEXT_ROW:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_ROW;
+      }
+      break;
+    case SKIP:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP)) {
+        return ReturnCode.SKIP;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.NEXT_COL)) {
+        return ReturnCode.NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_ROW;
+      }
+      break;
+    case NEXT_COL:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL, ReturnCode.SKIP,
+        ReturnCode.NEXT_COL)) {
+        return ReturnCode.NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_ROW;
+      }
+      break;
+    case NEXT_ROW:
+      return ReturnCode.NEXT_ROW;
+    }
+    throw new IllegalStateException(
+        "Received code is not valid. rc: " + rc + ", localRC: " + localRC);
+  }
+
+  private ReturnCode filterKeyValueWithMustPassAll(Cell c) throws IOException {
+    ReturnCode rc = ReturnCode.INCLUDE;
+    Cell transformed = c;
+    this.seekHintFilter.clear();
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (filter.filterAllRemaining()) {
+        return ReturnCode.NEXT_ROW;
+      }
+      ReturnCode localRC = filter.filterKeyValue(c);
+      rc = mergeReturnCode(rc, localRC);
+
+      // For INCLUDE* case, we need to update the transformed cell.
+      if (isInReturnCodes(localRC, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        transformed = filter.transformCell(transformed);
+      }
+      if (localRC == ReturnCode.SEEK_NEXT_USING_HINT) {
+        seekHintFilter.add(filter);
+      }
+    }
+    this.transformedCell = transformed;
+    if (!seekHintFilter.isEmpty()) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+    return rc;
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(Cell c) throws IOException {
+    if (isEmpty()) {
+      return ReturnCode.INCLUDE;
+    }
+    this.referenceCell = c;
+    return filterKeyValueWithMustPassAll(c);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      filters.get(i).reset();
+    }
+    seekHintFilter.clear();
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] rowKey, int offset, int length) throws 
IOException {
+    if (isEmpty()) {
+      return super.filterRowKey(rowKey, offset, length);
+    }
+    boolean retVal = false;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (filter.filterAllRemaining() || filter.filterRowKey(rowKey, offset, 
length)) {
+        retVal = true;
+      }
+    }
+    return retVal;
+  }
+
+  @Override
+  public boolean filterRowKey(Cell firstRowCell) throws IOException {
+    if (isEmpty()) {
+      return super.filterRowKey(firstRowCell);
+    }
+    boolean retVal = false;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (filter.filterAllRemaining() || filter.filterRowKey(firstRowCell)) {
+        retVal = true;
+      }
+    }
+    return retVal;
+  }
+
+  @Override
+  public boolean filterAllRemaining() throws IOException {
+    if (isEmpty()) {
+      return super.filterAllRemaining();
+    }
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      if (filters.get(i).filterAllRemaining()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public boolean filterRow() throws IOException {
+    if (isEmpty()) {
+      return super.filterRow();
+    }
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (filter.filterRow()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
+    if (isEmpty()) {
+      return super.getNextCellHint(currentCell);
+    }
+    Cell maxHint = null;
+    for (Filter filter : seekHintFilter) {
+      if (filter.filterAllRemaining()) {
+        continue;
+      }
+      Cell curKeyHint = filter.getNextCellHint(currentCell);
+      if (maxHint == null) {
+        maxHint = curKeyHint;
+        continue;
+      }
+      if (this.compareCell(maxHint, curKeyHint) < 0) {
+        maxHint = curKeyHint;
+      }
+    }
+    return maxHint;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
new file mode 100644
index 0000000..55306f2
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
@@ -0,0 +1,383 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * FilterListWithOR represents an ordered list of filters which will be 
evaluated with an OR
+ * operator.
+ */
+@InterfaceAudience.Private
+public class FilterListWithOR extends FilterListBase {
+
+  /**
+   * Save previous return code and previous cell for every filter in filter 
list. For MUST_PASS_ONE,
+   * we use the previous return code to decide whether we should pass current 
cell encountered to
+   * the filter. For MUST_PASS_ALL, the two list are meaningless.
+   */
+  private List<ReturnCode> prevFilterRCList = null;
+  private List<Cell> prevCellList = null;
+
+  public FilterListWithOR(List<Filter> filters) {
+    super(filters);
+    prevFilterRCList = new ArrayList<>(Collections.nCopies(filters.size(), 
null));
+    prevCellList = new ArrayList<>(Collections.nCopies(filters.size(), null));
+  }
+
+  @Override
+  public void addFilterLists(List<Filter> filters) {
+    if (checkAndGetReversed(filters, isReversed()) != isReversed()) {
+      throw new IllegalArgumentException("Filters in the list must have the 
same reversed flag");
+    }
+    this.filters.addAll(filters);
+    this.prevFilterRCList.addAll(Collections.nCopies(filters.size(), null));
+    this.prevCellList.addAll(Collections.nCopies(filters.size(), null));
+  }
+
+  @Override
+  protected String formatLogFilters(List<Filter> logFilters) {
+    return String.format("FilterList OR (%d/%d): %s", logFilters.size(), 
this.size(),
+      logFilters.toString());
+  }
+
+  /**
+   * For MUST_PASS_ONE, we cannot make sure that when filter-A in filter list 
return NEXT_COL then
+   * the next cell passing to filterList will be the first cell in next 
column, because if filter-B
+   * in filter list return SKIP, then the filter list will return SKIP. In 
this case, we should pass
+   * the cell following the previous cell, and it's possible that the next 
cell has the same column
+   * as the previous cell even if filter-A has NEXT_COL returned for the 
previous cell. So we should
+   * save the previous cell and the return code list when checking previous 
cell for every filter in
+   * filter list, and verify if currentCell fit the previous return code, if 
fit then pass the
+   * currentCell to the corresponding filter. (HBASE-17678)
+   */
+  private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell 
currentCell, int filterIdx)
+      throws IOException {
+    ReturnCode prevCode = this.prevFilterRCList.get(filterIdx);
+    if (prevCell == null || prevCode == null) {
+      return true;
+    }
+    switch (prevCode) {
+    case INCLUDE:
+    case SKIP:
+      return true;
+    case SEEK_NEXT_USING_HINT:
+      Cell nextHintCell = getNextCellHint(prevCell);
+      return nextHintCell == null || this.compareCell(currentCell, 
nextHintCell) >= 0;
+    case NEXT_COL:
+    case INCLUDE_AND_NEXT_COL:
+      return !CellUtil.matchingRowColumn(prevCell, currentCell);
+    case NEXT_ROW:
+    case INCLUDE_AND_SEEK_NEXT_ROW:
+      return !CellUtil.matchingRows(prevCell, currentCell);
+    default:
+      throw new IllegalStateException("Received code is not valid.");
+    }
+  }
+
+  /**
+   * FilterList with MUST_PASS_ONE choose the minimal forward step among 
sub-filter in filter list.
+   * Let's call it: The Minimal Step Rule. So if filter-A in filter list 
return INCLUDE and filter-B
+   * in filter list return INCLUDE_AND_NEXT_COL, then the filter list should 
return INCLUDE. For
+   * SEEK_NEXT_USING_HINT, it's more special, because we do not know how far 
it will forward, so we
+   * use SKIP by default.<br/>
+   * <br/>
+   * The jump step will be:
+   *
+   * <pre>
+   * INCLUDE &lt; SKIP &lt; INCLUDE_AND_NEXT_COL &lt; NEXT_COL &lt; 
INCLUDE_AND_SEEK_NEXT_ROW &lt; NEXT_ROW &lt; SEEK_NEXT_USING_HINT
+   * </pre>
+   *
+   * Here, we have the following map to describe The Minimal Step Rule. if 
current return code (for
+   * previous sub-filters in filter list) is <strong>ReturnCode</strong>, and 
current filter returns
+   * <strong>localRC</strong>, then we should return map[ReturnCode][localRC] 
for the merged result,
+   * according to The Minimal Step Rule.<br/>
+   *
+   * <pre>
+   * LocalCode\ReturnCode       INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW             
     SEEK_NEXT_USING_HINT
+   * INCLUDE                    INCLUDE INCLUDE                  INCLUDE       
             INCLUDE   INCLUDE               INCLUDE                   INCLUDE
+   * INCLUDE_AND_NEXT_COL       INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_NEXT_COL       INCLUDE   INCLUDE_AND_NEXT_COL  INCLUDE_AND_NEXT_COL 
     INCLUDE
+   * INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  INCLUDE   INCLUDE_AND_NEXT_COL  
INCLUDE_AND_SEEK_NEXT_ROW INCLUDE
+   * SKIP                       INCLUDE INCLUDE                  INCLUDE       
             SKIP      SKIP                  SKIP                      SKIP
+   * NEXT_COL                   INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_NEXT_COL       SKIP      NEXT_COL              NEXT_COL             
     SKIP
+   * NEXT_ROW                   INCLUDE INCLUDE_AND_NEXT_COL     
INCLUDE_AND_SEEK_NEXT_ROW  SKIP      NEXT_COL              NEXT_ROW             
     SKIP
+   * SEEK_NEXT_USING_HINT       INCLUDE INCLUDE                  INCLUDE       
             SKIP      SKIP                  SKIP                      
SEEK_NEXT_USING_HINT
+   * </pre>
+   *
+   * @param rc Return code which is calculated by previous sub-filter(s) in 
filter list.
+   * @param localRC Return code of the current sub-filter in filter list.
+   * @return Return code which is merged by the return code of previous 
sub-filter(s) and the return
+   *         code of current sub-filter.
+   */
+  private ReturnCode mergeReturnCode(ReturnCode rc, ReturnCode localRC) {
+    if (rc == null) return localRC;
+    switch (localRC) {
+    case INCLUDE:
+      return ReturnCode.INCLUDE;
+    case INCLUDE_AND_NEXT_COL:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP,
+        ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW,
+        ReturnCode.NEXT_COL, ReturnCode.NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+      break;
+    case INCLUDE_AND_SEEK_NEXT_ROW:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, ReturnCode.SKIP,
+        ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL, 
ReturnCode.NEXT_COL)) {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW, 
ReturnCode.NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
+      }
+      break;
+    case SKIP:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW,
+        ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.SKIP;
+      }
+      break;
+    case NEXT_COL:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, 
ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.SKIP;
+      }
+      break;
+    case NEXT_ROW:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_NEXT_COL)) {
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, 
ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.SKIP;
+      }
+      if (isInReturnCodes(rc, ReturnCode.NEXT_COL)) {
+        return ReturnCode.NEXT_COL;
+      }
+      if (isInReturnCodes(rc, ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_ROW;
+      }
+      break;
+    case SEEK_NEXT_USING_HINT:
+      if (isInReturnCodes(rc, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        return ReturnCode.INCLUDE;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SKIP, ReturnCode.NEXT_COL, 
ReturnCode.NEXT_ROW)) {
+        return ReturnCode.SKIP;
+      }
+      if (isInReturnCodes(rc, ReturnCode.SEEK_NEXT_USING_HINT)) {
+        return ReturnCode.SEEK_NEXT_USING_HINT;
+      }
+      break;
+    }
+    throw new IllegalStateException(
+        "Received code is not valid. rc: " + rc + ", localRC: " + localRC);
+  }
+
+  private void updatePrevFilterRCList(int index, ReturnCode currentRC) {
+    prevFilterRCList.set(index, currentRC);
+  }
+
+  private void updatePrevCellList(int index, Cell currentCell, ReturnCode 
currentRC) {
+    if (currentCell == null || currentRC == ReturnCode.INCLUDE || currentRC == 
ReturnCode.SKIP) {
+      // If previous return code is INCLUDE or SKIP, we should always pass the 
next cell to the
+      // corresponding sub-filter(need not test 
shouldPassCurrentCellToFilter() method), So we
+      // need not save current cell to prevCellList for saving heap memory.
+      prevCellList.set(index, null);
+    } else {
+      prevCellList.set(index, KeyValueUtil.toNewKeyCell(currentCell));
+    }
+  }
+
+  private ReturnCode filterKeyValueWithMustPassOne(Cell c) throws IOException {
+    ReturnCode rc = null;
+    boolean everyFilterReturnHint = true;
+    Cell transformed = c;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+
+      Cell prevCell = this.prevCellList.get(i);
+      if (filter.filterAllRemaining() || 
!shouldPassCurrentCellToFilter(prevCell, c, i)) {
+        everyFilterReturnHint = false;
+        continue;
+      }
+
+      ReturnCode localRC = filter.filterKeyValue(c);
+
+      // Update previous return code and previous cell for filter[i].
+      updatePrevFilterRCList(i, localRC);
+      updatePrevCellList(i, c, localRC);
+
+      if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
+        everyFilterReturnHint = false;
+      }
+
+      rc = mergeReturnCode(rc, localRC);
+
+      // For INCLUDE* case, we need to update the transformed cell.
+      if (isInReturnCodes(localRC, ReturnCode.INCLUDE, 
ReturnCode.INCLUDE_AND_NEXT_COL,
+        ReturnCode.INCLUDE_AND_SEEK_NEXT_ROW)) {
+        transformed = filter.transformCell(transformed);
+      }
+    }
+
+    this.transformedCell = transformed;
+    if (everyFilterReturnHint) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    } else if (rc == null) {
+      // Each sub-filter in filter list got true for filterAllRemaining().
+      return ReturnCode.SKIP;
+    } else {
+      return rc;
+    }
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(Cell c) throws IOException {
+    if (isEmpty()) {
+      return ReturnCode.INCLUDE;
+    }
+    this.referenceCell = c;
+    return filterKeyValueWithMustPassOne(c);
+  }
+
+  @Override
+  public void reset() throws IOException {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      filters.get(i).reset();
+      prevFilterRCList.set(i, null);
+      prevCellList.set(i, null);
+    }
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] rowKey, int offset, int length) throws 
IOException {
+    if (isEmpty()) {
+      return super.filterRowKey(rowKey, offset, length);
+    }
+    boolean retVal = true;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (!filter.filterAllRemaining() && !filter.filterRowKey(rowKey, offset, 
length)) {
+        retVal = false;
+      }
+    }
+    return retVal;
+  }
+
+  @Override
+  public boolean filterRowKey(Cell firstRowCell) throws IOException {
+    if (isEmpty()) {
+      return super.filterRowKey(firstRowCell);
+    }
+    boolean retVal = true;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (!filter.filterAllRemaining() && !filter.filterRowKey(firstRowCell)) {
+        retVal = false;
+      }
+    }
+    return retVal;
+  }
+
+  @Override
+  public boolean filterAllRemaining() throws IOException {
+    if (isEmpty()) {
+      return super.filterAllRemaining();
+    }
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      if (!filters.get(i).filterAllRemaining()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean filterRow() throws IOException {
+    if (isEmpty()) {
+      return super.filterRow();
+    }
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+      if (!filter.filterRow()) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
+    if (isEmpty()) {
+      return super.getNextCellHint(currentCell);
+    }
+    Cell minKeyHint = null;
+    // If any condition can pass, we need to keep the min hint
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      if (filters.get(i).filterAllRemaining()) {
+        continue;
+      }
+      Cell curKeyHint = filters.get(i).getNextCellHint(currentCell);
+      if (curKeyHint == null) {
+        // If we ever don't have a hint and this is must-pass-one, then no hint
+        return null;
+      }
+      // If this is the first hint we find, set it
+      if (minKeyHint == null) {
+        minKeyHint = curKeyHint;
+        continue;
+      }
+      if (this.compareCell(minKeyHint, curKeyHint) > 0) {
+        minKeyHint = curKeyHint;
+      }
+    }
+    return minKeyHint;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/b32bff02/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
index f7cba18..042ca08 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.testclassification.FilterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.Assert;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -799,5 +800,93 @@ public class TestFilterList {
     filterList = new FilterList(Operator.MUST_PASS_ONE, filter6, filter6);
     assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, 
filterList.filterKeyValue(kv1));
   }
+
+  static class MockSeekHintFilter extends FilterBase {
+    private Cell returnCell;
+
+    public MockSeekHintFilter(Cell returnCell) {
+      this.returnCell = returnCell;
+    }
+
+    @Override
+    public ReturnCode filterKeyValue(Cell v) throws IOException {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+
+    @Override
+    public Cell getNextCellHint(Cell currentCell) throws IOException {
+      return this.returnCell;
+    }
+  }
+
+  @Test
+  public void testReversedFilterListWithMockSeekHintFilter() throws 
IOException {
+    KeyValue kv1 = new KeyValue(Bytes.toBytes("row1"), Bytes.toBytes("fam"), 
Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    KeyValue kv2 = new KeyValue(Bytes.toBytes("row2"), Bytes.toBytes("fam"), 
Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    KeyValue kv3 = new KeyValue(Bytes.toBytes("row3"), Bytes.toBytes("fam"), 
Bytes.toBytes("a"), 1,
+        Bytes.toBytes("value"));
+    Filter filter1 = new MockSeekHintFilter(kv1);
+    filter1.setReversed(true);
+    Filter filter2 = new MockSeekHintFilter(kv2);
+    filter2.setReversed(true);
+    Filter filter3 = new MockSeekHintFilter(kv3);
+    filter3.setReversed(true);
+
+    FilterList filterList = new FilterList(Operator.MUST_PASS_ONE);
+    filterList.setReversed(true);
+    filterList.addFilter(filter1);
+    filterList.addFilter(filter2);
+    filterList.addFilter(filter3);
+
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, 
filterList.filterKeyValue(kv1));
+    Assert.assertEquals(kv3, filterList.getNextCellHint(kv1));
+
+    filterList = new FilterList(Operator.MUST_PASS_ALL);
+    filterList.setReversed(true);
+    filterList.addFilter(filter1);
+    filterList.addFilter(filter2);
+    filterList.addFilter(filter3);
+
+    Assert.assertEquals(ReturnCode.SEEK_NEXT_USING_HINT, 
filterList.filterKeyValue(kv1));
+    Assert.assertEquals(kv1, filterList.getNextCellHint(kv1));
+  }
+
+  @Test
+  public void testReversedFilterListWithOR() throws IOException {
+    byte[] r22 = Bytes.toBytes("Row22");
+    byte[] r2 = Bytes.toBytes("Row2");
+    byte[] r1 = Bytes.toBytes("Row1");
+
+    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+    filterList.setReversed(true);
+    PrefixFilter prefixFilter = new PrefixFilter(r2);
+    prefixFilter.setReversed(true);
+    filterList.addFilter(prefixFilter);
+    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new 
KeyValue(r22, r22, r22)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new 
KeyValue(r2, r2, r2)));
+
+    filterList.reset();
+    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
+    assertEquals(ReturnCode.SKIP, filterList.filterKeyValue(new KeyValue(r1, 
r1, r1)));
+
+    filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+    filterList.setReversed(true);
+    AlwaysNextColFilter alwaysNextColFilter = new AlwaysNextColFilter();
+    alwaysNextColFilter.setReversed(true);
+    prefixFilter = new PrefixFilter(r2);
+    prefixFilter.setReversed(true);
+    filterList.addFilter(alwaysNextColFilter);
+    filterList.addFilter(prefixFilter);
+    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r22));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new 
KeyValue(r22, r22, r22)));
+    assertEquals(ReturnCode.INCLUDE, filterList.filterKeyValue(new 
KeyValue(r2, r2, r2)));
+
+    filterList.reset();
+    filterList.filterRowKey(KeyValueUtil.createFirstOnRow(r1));
+    assertEquals(ReturnCode.NEXT_COL, filterList.filterKeyValue(new 
KeyValue(r1, r1, r1)));
+  }
 }
 

Reply via email to