You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/03/08 00:41:15 UTC

hbase git commit: HBASE-15243 Utilize the lowest seek value when all Filters in MUST_PASS_ONE FilterList return SEEK_NEXT_USING_HINT

Repository: hbase
Updated Branches:
  refs/heads/master 05161fcbf -> 05c1309b3


HBASE-15243 Utilize the lowest seek value when all Filters in MUST_PASS_ONE FilterList return SEEK_NEXT_USING_HINT


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

Branch: refs/heads/master
Commit: 05c1309b3c4ef5eb2634a84339f608fc4e694318
Parents: 05161fc
Author: tedyu <yu...@gmail.com>
Authored: Mon Mar 7 15:41:06 2016 -0800
Committer: tedyu <yu...@gmail.com>
Committed: Mon Mar 7 15:41:06 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/filter/FilterList.java  |  25 ++-
 .../TestFilterListOrOperatorWithBlkCnt.java     | 175 +++++++++++++++++++
 2 files changed, 199 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/05c1309b/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 8ba1ccb..da7a084 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
@@ -254,6 +254,15 @@ final public class FilterList extends Filter {
     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) {
@@ -279,7 +288,11 @@ final public class FilterList extends Filter {
           continue;
         }
 
-        switch (filter.filterKeyValue(c)) {
+        ReturnCode localRC = filter.filterKeyValue(c);
+        if (localRC != ReturnCode.SEEK_NEXT_USING_HINT) {
+          seenNonHintReturnCode = true;
+        }
+        switch (localRC) {
         case INCLUDE:
           if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
             rc = ReturnCode.INCLUDE;
@@ -308,6 +321,13 @@ final public class FilterList extends Filter {
     // Save the transformed Cell for transform():
     this.transformedCell = transformed;
 
+    /*
+     * 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;
   }
 
@@ -423,6 +443,9 @@ final public class FilterList extends Filter {
     // 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(currentCell);
       if (curKeyHint == null) {
         // If we ever don't have a hint and this is must-pass-one, then no hint

http://git-wip-us.apache.org/repos/asf/hbase/blob/05c1309b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java
new file mode 100644
index 0000000..058280d
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterListOrOperatorWithBlkCnt.java
@@ -0,0 +1,175 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.MultiRowRangeFilter.RowRange;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/*
+ * This test is for the optimization added in HBASE-15243.
+ * FilterList with two MultiRowRangeFilter's is constructed using Operator.MUST_PASS_ONE.
+ */
+@Category(MediumTests.class)
+public class TestFilterListOrOperatorWithBlkCnt {
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final Log LOG = LogFactory.getLog(TestFilterListOrOperatorWithBlkCnt.class);
+  private byte[] family = Bytes.toBytes("family");
+  private byte[] qf = Bytes.toBytes("qf");
+  private byte[] value = Bytes.toBytes("val");
+  private TableName tableName;
+  private int numRows = 10000;
+
+  /**
+   * @throws Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    long blkSize = 4096;
+    /*
+     * dfs block size is adjusted so that the specified number of rows would result in
+     * multiple blocks (8 for this test).
+     * Later in the test, assertion is made on the number of blocks read.
+     */
+    TEST_UTIL.getConfiguration().setLong("dfs.blocksize", blkSize);
+    TEST_UTIL.getConfiguration().setLong("dfs.bytes-per-checksum", blkSize);
+    TEST_UTIL.startMiniCluster();
+  }
+
+  /**
+   * @throws Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  private static long getBlkAccessCount() {
+    return HFile.dataBlockReadCnt.get();
+  }
+
+  @Test
+  public void testMultiRowRangeWithFilterListOrOperatorWithBlkCnt() throws IOException {
+    tableName = TableName.valueOf("TestMultiRowRangeFilterWithFilterListOrOperatorWithBlkCnt");
+    Table ht = TEST_UTIL.createTable(tableName, family, Integer.MAX_VALUE);
+    generateRows(numRows, ht, family, qf, value);
+
+    Scan scan = new Scan();
+    scan.setMaxVersions();
+    long blocksStart = getBlkAccessCount();
+
+    List<RowRange> ranges1 = new ArrayList<RowRange>();
+    ranges1.add(new RowRange(Bytes.toBytes(10), true, Bytes.toBytes(15), false));
+    ranges1.add(new RowRange(Bytes.toBytes(9980), true, Bytes.toBytes(9985), false));
+
+    MultiRowRangeFilter filter1 = new MultiRowRangeFilter(ranges1);
+
+    List<RowRange> ranges2 = new ArrayList<RowRange>();
+    ranges2.add(new RowRange(Bytes.toBytes(15), true, Bytes.toBytes(20), false));
+    ranges2.add(new RowRange(Bytes.toBytes(9985), true, Bytes.toBytes(9990), false));
+
+    MultiRowRangeFilter filter2 = new MultiRowRangeFilter(ranges2);
+
+    FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
+    filterList.addFilter(filter1);
+    filterList.addFilter(filter2);
+    scan.setFilter(filterList);
+    int resultsSize = getResultsSize(ht, scan);
+    LOG.info("found " + resultsSize + " results");
+    List<Cell> results1 = getScanResult(Bytes.toBytes(10), Bytes.toBytes(20), ht);
+    List<Cell> results2 = getScanResult(Bytes.toBytes(9980), Bytes.toBytes(9990), ht);
+
+    assertEquals(results1.size() + results2.size(), resultsSize);
+    long blocksEnd = getBlkAccessCount();
+    long diff = blocksEnd - blocksStart;
+    LOG.info("Diff in number of blocks " + diff);
+    /*
+     * Verify that we don't read all the blocks (8 in total).
+     */
+    assertEquals(4, diff);
+
+    ht.close();
+  }
+
+  private void generateRows(int numberOfRows, Table ht, byte[] family, byte[] qf, byte[] value)
+      throws IOException {
+    for (int i = 0; i < numberOfRows; i++) {
+      byte[] row = Bytes.toBytes(i);
+      Put p = new Put(row);
+      p.addColumn(family, qf, value);
+      p.setDurability(Durability.SKIP_WAL);
+      ht.put(p);
+    }
+    TEST_UTIL.flush();
+  }
+
+  private List<Cell> getScanResult(byte[] startRow, byte[] stopRow, Table ht) throws IOException {
+    Scan scan = new Scan();
+    scan.setMaxVersions();
+    if(!Bytes.toString(startRow).isEmpty()) {
+      scan.setStartRow(startRow);
+    }
+    if(!Bytes.toString(stopRow).isEmpty()) {
+      scan.setStopRow(stopRow);
+    }
+    ResultScanner scanner = ht.getScanner(scan);
+    List<Cell> kvList = new ArrayList<Cell>();
+    Result r;
+    while ((r = scanner.next()) != null) {
+      for (Cell kv : r.listCells()) {
+        kvList.add(kv);
+      }
+    }
+    return kvList;
+  }
+
+  private int getResultsSize(Table ht, Scan scan) throws IOException {
+    ResultScanner scanner = ht.getScanner(scan);
+    List<Cell> results = new ArrayList<Cell>();
+    Result r;
+    while ((r = scanner.next()) != null) {
+      for (Cell kv : r.listCells()) {
+        results.add(kv);
+      }
+    }
+    return results.size();
+  }
+}