You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/11/07 23:30:37 UTC

[8/8] hbase git commit: HBASE-18993 Backport patches in HBASE-18410 to branch-1.x branches

HBASE-18993 Backport patches in HBASE-18410 to branch-1.x branches

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 264cf0d4739ae9ad4343755d654efd73a9e623e0
Parents: c5ab70d
Author: huzheng <op...@gmail.com>
Authored: Thu Oct 19 21:51:18 2017 +0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Nov 7 14:00:27 2017 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/filter/Filter.java  |  14 +-
 .../apache/hadoop/hbase/filter/FilterList.java  | 421 +++---------
 .../hadoop/hbase/filter/FilterListBase.java     | 177 +++++
 .../hadoop/hbase/filter/FilterListWithAND.java  | 258 ++++++++
 .../hadoop/hbase/filter/FilterListWithOR.java   | 380 +++++++++++
 .../hadoop/hbase/filter/TestFilterList.java     | 658 +++++++++++++++----
 .../hbase/filter/TestFilterListOnMini.java      |   6 +-
 7 files changed, 1464 insertions(+), 450 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/264cf0d4/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
index 0a7a184..a22b5c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
@@ -166,15 +166,23 @@ public abstract class Filter {
      */
     NEXT_COL,
     /**
-     * Done with columns, skip to next row. Note that filterRow() will
-     * still be called.
+     * Seek to next row in current family. It may still pass a cell whose family is different but
+     * row is the same as previous cell to {@link #filterKeyValue(Cell)} , even if we get a NEXT_ROW
+     * returned for previous cell. For more details see HBASE-18368. <br>
+     * Once reset() method was invoked, then we switch to the next row for all family, and you can
+     * catch the event by invoking CellUtils.matchingRows(previousCell, currentCell). <br>
+     * Note that filterRow() will still be called. <br>
      */
     NEXT_ROW,
     /**
      * Seek to next key which is given as hint by the filter.
      */
     SEEK_NEXT_USING_HINT,
-}
+    /**
+     * Include KeyValue and done with row, seek to next. See NEXT_ROW
+     */
+    INCLUDE_AND_SEEK_NEXT_ROW,
+  }
 
   /**
    * Chance to alter the list of Cells to be submitted. Modifications to the list will carry on

http://git-wip-us.apache.org/repos/asf/hbase/blob/264cf0d4/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 be22e5d..0b39b56 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
@@ -21,123 +21,99 @@ package org.apache.hadoop.hbase.filter;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.classification.InterfaceStability;
+import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-
 /**
- * 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
 @InterfaceStability.Stable
 final public class FilterList extends Filter {
+
   /** set operator */
   @InterfaceAudience.Public
   @InterfaceStability.Stable
-  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 List<Filter> filters = new ArrayList<Filter>();
-  private Filter seekHintFilter = null;
-
-  /** Reference Cell used by {@link #transformCell(Cell)} for validation purpose. */
-  private Cell referenceKV = 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 transformedKV = null;
+  private FilterListBase filterListBase;
 
   /**
-   * Constructor that takes a set of {@link Filter}s. The default operator
-   * MUST_PASS_ALL is assumed.
-   *
-   * @param rowFilters list of filters
+   * Constructor that takes a set of {@link Filter}s and an operator.
+   * @param operator Operator to process filter set with.
+   * @param filters Set of row filters.
    */
-  public FilterList(final List<Filter> rowFilters) {
-    if (rowFilters instanceof ArrayList) {
-      this.filters = 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);
     } else {
-      this.filters = new ArrayList<Filter>(rowFilters);
+      throw new IllegalArgumentException("Invalid operator: " + operator);
     }
+    this.operator = operator;
   }
 
   /**
-   * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
-   * MUST_PASS_ALL is assumed.
-   * @param rowFilters
+   * 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 filters list of filters
    */
-  public FilterList(final Filter... rowFilters) {
-    this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
+  public FilterList(final List<Filter> filters) {
+    this(Operator.MUST_PASS_ALL, filters);
   }
 
   /**
-   * Constructor that takes an operator.
-   *
-   * @param operator Operator to process filter set with.
+   * Constructor that takes a var arg number of {@link Filter}s. The default operator MUST_PASS_ALL
+   * is assumed.
+   * @param filters
    */
-  public FilterList(final Operator operator) {
-    this.operator = operator;
+  public FilterList(final Filter... filters) {
+    this(Operator.MUST_PASS_ALL, Arrays.asList(filters));
   }
 
   /**
-   * Constructor that takes a set of {@link Filter}s and an operator.
-   *
+   * Constructor that takes an operator.
    * @param operator Operator to process filter set with.
-   * @param rowFilters Set of row filters.
    */
-  public FilterList(final Operator operator, final List<Filter> rowFilters) {
-    this.filters = new ArrayList<Filter>(rowFilters);
-    this.operator = operator;
+  public FilterList(final Operator operator) {
+    this(operator, new ArrayList<Filter>());
   }
 
   /**
    * 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.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
-    this.operator = operator;
+  public FilterList(final Operator operator, final Filter... filters) {
+    this(operator, Arrays.asList(filters));
   }
 
   /**
    * Get the operator.
-   *
    * @return operator
    */
   public Operator getOperator() {
@@ -146,245 +122,110 @@ final public class FilterList extends Filter {
 
   /**
    * Get the filters.
-   *
    * @return filters
    */
   public List<Filter> getFilters() {
-    return filters;
+    return filterListBase.getFilters();
+  }
+
+  public int size() {
+    return filterListBase.size();
+  }
+
+  public void addFilter(List<Filter> filters) {
+    filterListBase.addFilterLists(filters);
   }
 
   /**
    * Add a filter.
-   *
    * @param filter another filter
    */
   public void addFilter(Filter filter) {
-    if (this.isReversed() != filter.isReversed()) {
-      throw new IllegalArgumentException(
-          "Filters in the list must have the same reversed flag, this.reversed="
-              + this.isReversed());
-    }
-    this.filters.add(filter);
+    addFilter(Collections.singletonList(filter));
   }
 
   @Override
   public void reset() throws IOException {
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
-      filters.get(i).reset();
-    }
-    seekHintFilter = null;
+    filterListBase.reset();
   }
 
   @Override
   public boolean filterRowKey(byte[] rowKey, int offset, int length) throws IOException {
-    boolean flag = (this.operator == Operator.MUST_PASS_ONE) ? true : false;
-    int listize = filters.size();
-    for (int i = 0; i < listize; 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 filterAllRemaining() throws IOException {
-    int listize = filters.size();
-    for (int i = 0; i < listize; 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 v) throws IOException {
-    // transformCell() is expected to follow an inclusive filterKeyValue() immediately:
-    if (!v.equals(this.referenceKV)) {
-      throw new IllegalStateException("Reference Cell: " + this.referenceKV + " does not match: "
-          + v);
-    }
-    return this.transformedKV;
+  public Cell transformCell(Cell c) throws IOException {
+    return filterListBase.transformCell(c);
   }
 
   /**
-   * WARNING: please to not override this method.  Instead override {@link #transformCell(Cell)}.
-   *
-   * When removing this, its body should be placed in transformCell.
-   *
-   * This is for transition from 0.94 -&gt; 0.96
+   * WARNING: please to not override this method. Instead override {@link #transformCell(Cell)}.
+   * When removing this, its body should be placed in transformCell. This is for transition from
+   * 0.94 -&gt; 0.96
    */
   @Deprecated
   @Override
   public KeyValue transform(KeyValue v) throws IOException {
-    // transform() is expected to follow an inclusive filterKeyValue() immediately:
-    if (!v.equals(this.referenceKV)) {
-      throw new IllegalStateException(
-          "Reference Cell: " + this.referenceKV + " does not match: " + v);
-     }
-    return KeyValueUtil.ensureKeyValue(this.transformedKV);
+    return KeyValueUtil.ensureKeyValue(transformCell((Cell) v));
   }
 
-  
-  @Override
-  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
-    justification="Intentional")
-  public ReturnCode filterKeyValue(Cell v) throws IOException {
-    this.referenceKV = v;
-
-    // Accumulates successive transformation of every filter that includes the Cell:
-    Cell transformed = v;
-
-    ReturnCode rc = operator == Operator.MUST_PASS_ONE?
-        ReturnCode.SKIP: ReturnCode.INCLUDE;
-    int listize = filters.size();
-    /*
-     * When all filters in a MUST_PASS_ONE FilterList return a SEEK_USING_NEXT_HINT code,
-     * we should return SEEK_NEXT_USING_HINT from the FilterList to utilize the lowest seek value.
-     * 
-     * The following variable tracks whether any of the Filters returns ReturnCode other than
-     * SEEK_NEXT_USING_HINT for MUST_PASS_ONE FilterList, in which case the optimization would
-     * be skipped.
-     */
-    boolean seenNonHintReturnCode = false;
-    for (int i = 0; i < listize; i++) {
-      Filter filter = filters.get(i);
-      if (operator == Operator.MUST_PASS_ALL) {
-        if (filter.filterAllRemaining()) {
-          return ReturnCode.NEXT_ROW;
-        }
-        ReturnCode code = filter.filterKeyValue(v);
-        switch (code) {
-        // Override INCLUDE and continue to evaluate.
-        case INCLUDE_AND_NEXT_COL:
-          rc = ReturnCode.INCLUDE_AND_NEXT_COL; // FindBugs SF_SWITCH_FALLTHROUGH
-        case INCLUDE:
-          transformed = filter.transformCell(transformed);
-          continue;
-        case SEEK_NEXT_USING_HINT:
-          seekHintFilter = filter;
-          return code;
-        default:
-          return code;
-        }
-      } else if (operator == Operator.MUST_PASS_ONE) {
-        if (filter.filterAllRemaining()) {
-          seenNonHintReturnCode = true;
-          continue;
-        }
-
-        ReturnCode localRC = filter.filterKeyValue(v);
-        if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
-          seenNonHintReturnCode = true;
-        }
-        switch (localRC) {
-        case INCLUDE:
-          if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
-            rc = ReturnCode.INCLUDE;
-          }
-          transformed = filter.transformCell(transformed);
-          break;
-        case INCLUDE_AND_NEXT_COL:
-          rc = ReturnCode.INCLUDE_AND_NEXT_COL;
-          transformed = filter.transformCell(transformed);
-          // must continue here to evaluate all filters
-          break;
-        case NEXT_ROW:
-          break;
-        case SKIP:
-          break;
-        case NEXT_COL:
-          break;
-        case SEEK_NEXT_USING_HINT:
-          break;
-        default:
-          throw new IllegalStateException("Received code is not valid.");
-        }
-      }
-    }
-
-    // Save the transformed Cell for transform():
-    this.transformedKV = transformed;
+  /**
+   * Internal implementation of {@link #filterKeyValue(Cell)}. Compared to the
+   * {@link #filterKeyValue(Cell)} method, this method accepts an additional parameter named
+   * transformedCell. This parameter indicates the initial value of transformed cell before this
+   * filter operation. <br/>
+   * For FilterList, we can consider a filter list as a node in a tree. sub-filters of the filter
+   * list are children of the relative node. The logic of transforming cell of a filter list, well,
+   * we can consider it as the process of post-order tree traverse. For a node , Before we traverse
+   * the current child, we should set the traverse result (transformed cell) of previous node(s) as
+   * the initial value. so the additional currentTransformedCell parameter is needed (HBASE-18879).
+   * @param c The cell in question.
+   * @param transformedCell The transformed cell of previous filter(s)
+   * @return ReturnCode of this filter operation.
+   * @throws IOException
+   */
+  ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException {
+    return this.filterListBase.internalFilterKeyValue(c, transformedCell);
+  }
 
-    /*
-     * The seenNonHintReturnCode flag is intended only for Operator.MUST_PASS_ONE branch.
-     * If we have seen non SEEK_NEXT_USING_HINT ReturnCode, respect that ReturnCode.
-     */
-    if (operator == Operator.MUST_PASS_ONE && !seenNonHintReturnCode) {
-      return ReturnCode.SEEK_NEXT_USING_HINT;
-    }
-    return rc;
+  @Override
+  public ReturnCode filterKeyValue(Cell c) throws IOException {
+    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 {
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
-      filters.get(i).filterRowCells(cells);
-    }
+    filterListBase.filterRowCells(cells);
   }
 
   @Override
   public boolean hasFilterRow() {
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
-      if (filters.get(i).hasFilterRow()) {
-        return true;
-      }
-    }
-    return false;
+    return filterListBase.hasFilterRow();
   }
 
   @Override
   public boolean filterRow() throws IOException {
-    int listize = filters.size();
-    for (int i = 0; i < listize; 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();
   }
 
   /**
    * @return The filter serialized using pb
    */
   public byte[] toByteArray() throws IOException {
-    FilterProtos.FilterList.Builder builder =
-      FilterProtos.FilterList.newBuilder();
+    FilterProtos.FilterList.Builder builder = FilterProtos.FilterList.newBuilder();
     builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
+    ArrayList<Filter> filters = filterListBase.getFilters();
+    for (int i = 0, n = filters.size(); i < n; i++) {
       builder.addFilters(ProtobufUtil.toFilter(filters.get(i)));
     }
     return builder.build().toByteArray();
@@ -396,8 +237,7 @@ final public class FilterList extends Filter {
    * @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);
@@ -407,105 +247,60 @@ final public class FilterList extends Filter {
 
     List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
     try {
-      List<org.apache.hadoop.hbase.protobuf.generated.FilterProtos.Filter> filtersList =
-          proto.getFiltersList();
-      int listSize = filtersList.size();
-      for (int i = 0; i < listSize; i++) {
+      List<FilterProtos.Filter> filtersList = proto.getFiltersList();
+      for (int i = 0, n = filtersList.size(); i < n; i++) {
         rowFilters.add(ProtobufUtil.toFilter(filtersList.get(i)));
       }
     } catch (IOException ioe) {
       throw new DeserializationException(ioe);
     }
-    return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
+    return new FilterList(Operator.valueOf(proto.getOperator().name()), rowFilters);
   }
 
   /**
    * @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
   @Deprecated
   public KeyValue getNextKeyHint(KeyValue currentKV) throws IOException {
-    return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell)currentKV));
+    return KeyValueUtil.ensureKeyValue(getNextCellHint((Cell) currentKV));
   }
 
   @Override
-  public Cell getNextCellHint(Cell currentKV) throws IOException {
-    Cell keyHint = null;
-    if (operator == Operator.MUST_PASS_ALL) {
-      keyHint = seekHintFilter.getNextCellHint(currentKV);
-      return keyHint;
-    }
-
-    // If any condition can pass, we need to keep the min hint
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
-      if (filters.get(i).filterAllRemaining()) {
-        continue;
-      }
-      Cell curKeyHint = filters.get(i).getNextCellHint(currentKV);
-      if (curKeyHint == null) {
-        // If we ever don't have a hint and this is must-pass-one, then no hint
-        return null;
-      }
-      if (curKeyHint != null) {
-        // If this is the first hint we find, set it
-        if (keyHint == null) {
-          keyHint = curKeyHint;
-          continue;
-        }
-        if (KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
-          keyHint = curKeyHint;
-        }
-      }
-    }
-    return keyHint;
+  public Cell getNextCellHint(Cell currentCell) throws IOException {
+    return this.filterListBase.getNextCellHint(currentCell);
   }
 
   @Override
   public boolean isFamilyEssential(byte[] name) throws IOException {
-    int listize = filters.size();
-    for (int i = 0; i < listize; i++) {
-      if (filters.get(i).isFamilyEssential(name)) {
-        return true;
-      }
-    }
-    return false;
+    return this.filterListBase.isFamilyEssential(name);
   }
 
   @Override
   public void setReversed(boolean reversed) {
-    int listize = filters.size();
-    for (int i = 0; i < listize; 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/264cf0d4/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..5f0d1df
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListBase.java
@@ -0,0 +1,177 @@
+/**
+ *
+ * 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.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.classification.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<Filter>(filters);
+  }
+
+  protected static boolean isInReturnCodes(ReturnCode testRC, ReturnCode... returnCodes) {
+    for (ReturnCode rc : returnCodes) {
+      if (testRC.equals(rc)) return true;
+    }
+    return false;
+  }
+
+  protected static boolean checkAndGetReversed(List<Filter> rowFilters, boolean defaultValue) {
+    if (rowFilters.isEmpty()) {
+      return defaultValue;
+    }
+    boolean retValue = rowFilters.get(0).isReversed();
+    for (Filter filter : rowFilters) {
+      if (filter.isReversed() != retValue) {
+        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 = KeyValue.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;
+  }
+
+  /**
+   * Internal implementation of {@link #filterKeyValue(Cell)}
+   * @param c The cell in question.
+   * @param transformedCell The transformed cell of previous filter(s)
+   * @return ReturnCode of this filter operation.
+   * @throws IOException
+   * @see org.apache.hadoop.hbase.filter.FilterList#internalFilterKeyValue(Cell, Cell)
+   */
+  abstract ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException;
+
+  @Override
+  public ReturnCode filterKeyValue(Cell c) throws IOException {
+    return internalFilterKeyValue(c, c);
+  }
+
+  /**
+   * 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/264cf0d4/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..ca2c149
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java
@@ -0,0 +1,258 @@
+/**
+ *
+ * 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.classification.InterfaceAudience;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * FilterListWithAND represents an ordered list of filters which will be evaluated with an AND
+ * operator.
+ */
+@InterfaceAudience.Private
+public class FilterListWithAND extends FilterListBase {
+
+  private List<Filter> seekHintFilters = new ArrayList<>();
+
+  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);
+  }
+
+  @Override
+  ReturnCode internalFilterKeyValue(Cell c, Cell transformedCell) throws IOException {
+    if (isEmpty()) {
+      return ReturnCode.INCLUDE;
+    }
+    ReturnCode rc = ReturnCode.INCLUDE;
+    Cell transformed = transformedCell;
+    this.referenceCell = 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;
+      if (filter instanceof FilterList) {
+        localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed);
+      } else {
+        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) {
+        seekHintFilters.add(filter);
+      }
+    }
+    this.transformedCell = transformed;
+    if (!seekHintFilters.isEmpty()) {
+      return ReturnCode.SEEK_NEXT_USING_HINT;
+    }
+    return rc;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      filters.get(i).reset();
+    }
+    seekHintFilters.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 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 : seekHintFilters) {
+      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/264cf0d4/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..47d792c
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java
@@ -0,0 +1,380 @@
+/**
+ *
+ * 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.hadoop.hbase.classification.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<ReturnCode>(Collections.nCopies(filters.size(), (ReturnCode) null));
+    prevCellList = new ArrayList<Cell>(Collections.nCopies(filters.size(), (Cell) 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(), (ReturnCode) null));
+    this.prevCellList.addAll(Collections.nCopies(filters.size(), (Cell) 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) <br>
+   * Note that: In StoreScanner level, NEXT_ROW will skip to the next row in current family, and in
+   * RegionScanner level, NEXT_ROW will skip to the next row in current family and switch to the
+   * next family for RegionScanner, INCLUDE_AND_NEXT_ROW is the same. so we should pass current cell
+   * to the filter, if row mismatch or row match but column family mismatch. (HBASE-18368)
+   * @see org.apache.hadoop.hbase.filter.Filter.ReturnCode
+   */
+  private boolean shouldPassCurrentCellToFilter(Cell prevCell, Cell currentCell,
+      ReturnCode prevCode) throws IOException {
+    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:
+      // Once row changed, reset() will clear prevCells, so we need not to compare their rows
+      // because rows are the same here.
+      return !CellUtil.matchingColumn(prevCell, currentCell);
+    case NEXT_ROW:
+    case INCLUDE_AND_SEEK_NEXT_ROW:
+      // As described above, rows are definitely the same, so we only compare the family.
+      return !CellUtil.matchingFamily(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.NEXT_COL, ReturnCode.NEXT_ROW)) {
+        return ReturnCode.NEXT_COL;
+      }
+      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.copyToNewKeyValue(currentCell));
+    }
+  }
+
+  @Override
+  ReturnCode internalFilterKeyValue(Cell c, Cell transformCell) throws IOException {
+    if (isEmpty()) {
+      return ReturnCode.INCLUDE;
+    }
+    ReturnCode rc = null;
+    boolean everyFilterReturnHint = true;
+    Cell transformed = transformCell;
+    this.referenceCell = c;
+    for (int i = 0, n = filters.size(); i < n; i++) {
+      Filter filter = filters.get(i);
+
+      Cell prevCell = this.prevCellList.get(i);
+      ReturnCode prevCode = this.prevFilterRCList.get(i);
+      if (filter.filterAllRemaining() || !shouldPassCurrentCellToFilter(prevCell, c, prevCode)) {
+        everyFilterReturnHint = false;
+        continue;
+      }
+
+      ReturnCode localRC;
+      if (filter instanceof FilterList) {
+        localRC = ((FilterList) filter).internalFilterKeyValue(c, transformed);
+      } else {
+        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 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 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;
+  }
+}