You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2018/03/02 06:35:34 UTC

[25/50] [abbrv] hbase git commit: HBASE-19863 java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used

HBASE-19863 java.lang.IllegalStateException: isDelete failed when SingleColumnValueFilter is used

Signed-off-by: Chia-Ping Tsai <ch...@gmail.com>
Signed-off-by: ramkrish86 <ra...@gmail.com>


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

Branch: refs/heads/HBASE-19064
Commit: 393ab302ab08b70a839ec87e75fcf4b165765db2
Parents: 96ebab7
Author: Sergey Soldatov <ss...@apache.org>
Authored: Tue Feb 13 22:08:11 2018 -0800
Committer: Josh Elser <el...@apache.org>
Committed: Wed Feb 28 13:58:37 2018 -0500

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/StoreScanner.java |   6 +
 .../hadoop/hbase/HBaseTestingUtility.java       |  41 +++--
 .../hbase/regionserver/TestIsDeleteFailure.java | 160 +++++++++++++++++++
 3 files changed, 195 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/393ab302/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
index 1624810..9f6a015 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
@@ -816,6 +816,12 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
         return false;
       }
     } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRowColumn(cell, nextCell));
+    // We need this check because it may happen that the new scanner that we get
+    // during heap.next() is requiring reseek due of fake KV previously generated for
+    // ROWCOL bloom filter optimization. See HBASE-19863 for more details
+    if (nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) {
+      return false;
+    }
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/393ab302/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 2bdfd2d..b48abc6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -1378,23 +1378,40 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param htd
-   * @param families
-   * @param splitKeys
+   * @param htd table descriptor
+   * @param families array of column families
+   * @param splitKeys array of split keys
    * @param c Configuration to use
    * @return A Table instance for the created table.
-   * @throws IOException
+   * @throws IOException if getAdmin or createTable fails
    */
   public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys,
       Configuration c) throws IOException {
+    // Disable blooms (they are on by default as of 0.95) but we disable them here because
+    // tests have hard coded counts of what to expect in block cache, etc., and blooms being
+    // on is interfering.
+    return createTable(htd, families, splitKeys, BloomType.NONE, HConstants.DEFAULT_BLOCKSIZE, c);
+  }
+
+  /**
+   * Create a table.
+   * @param htd table descriptor
+   * @param families array of column families
+   * @param splitKeys array of split keys
+   * @param type Bloom type
+   * @param blockSize block size
+   * @param c Configuration to use
+   * @return A Table instance for the created table.
+   * @throws IOException if getAdmin or createTable fails
+   */
+
+  public Table createTable(TableDescriptor htd, byte[][] families, byte[][] splitKeys,
+      BloomType type, int blockSize, Configuration c) throws IOException {
     TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(htd);
     for (byte[] family : families) {
-      // Disable blooms (they are on by default as of 0.95) but we disable them here because
-      // tests have hard coded counts of what to expect in block cache, etc., and blooms being
-      // on is interfering.
-      builder.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family)
-                              .setBloomFilterType(BloomType.NONE)
-                              .build());
+      builder.addColumnFamily(
+          ColumnFamilyDescriptorBuilder.newBuilder(family).setBloomFilterType(type)
+              .setBlocksize(blockSize).build());
     }
     TableDescriptor td = builder.build();
     getAdmin().createTable(td, splitKeys);
@@ -1406,8 +1423,8 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
 
   /**
    * Create a table.
-   * @param htd
-   * @param splitRows
+   * @param htd table descriptor
+   * @param splitRows array of split keys
    * @return A Table instance for the created table.
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/393ab302/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
new file mode 100644
index 0000000..1198867
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestIsDeleteFailure.java
@@ -0,0 +1,160 @@
+/**
+ * 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.regionserver;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.filter.BinaryComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+/**
+ * Test failure in ScanDeleteTracker.isDeleted when ROWCOL bloom filter
+ * is used during a scan with a filter.
+ */
+@Category({ RegionServerTests.class, FilterTests.class, MediumTests.class })
+public class TestIsDeleteFailure {
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestIsDeleteFailure.class);
+
+  @Rule public TestName name = new TestName();
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.getConfiguration().setInt("hbase.regionserver.msginterval", 100);
+    TEST_UTIL.getConfiguration().setInt("hbase.client.pause", 250);
+    TEST_UTIL.getConfiguration().setInt("hbase.client.retries.number", 2);
+    TEST_UTIL.getConfiguration().setBoolean("hbase.master.enabletable.roundrobin", true);
+    TEST_UTIL.startMiniCluster(1);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void testIsDeleteFailure() throws Exception {
+    final HTableDescriptor table = new HTableDescriptor(TableName.valueOf(name.getMethodName()));
+    final byte[] family = Bytes.toBytes("0");
+    final byte[] c1 = Bytes.toBytes("C01");
+    final byte[] c2 = Bytes.toBytes("C02");
+    final byte[] c3 = Bytes.toBytes("C03");
+    final byte[] c4 = Bytes.toBytes("C04");
+    final byte[] c5 = Bytes.toBytes("C05");
+    final byte[] c6 = Bytes.toBytes("C07");
+    final byte[] c7 = Bytes.toBytes("C07");
+    final byte[] c8 = Bytes.toBytes("C08");
+    final byte[] c9 = Bytes.toBytes("C09");
+    final byte[] c10 = Bytes.toBytes("C10");
+    final byte[] c11 = Bytes.toBytes("C11");
+    final byte[] c12 = Bytes.toBytes("C12");
+    final byte[] c13 = Bytes.toBytes("C13");
+    final byte[] c14 = Bytes.toBytes("C14");
+    final byte[] c15 = Bytes.toBytes("C15");
+
+    final byte[] val = Bytes.toBytes("foo");
+    List<byte[]> fams = new ArrayList<>(1);
+    fams.add(family);
+    Table ht = TEST_UTIL
+        .createTable(table, fams.toArray(new byte[0][]), null, BloomType.ROWCOL, 10000,
+            new Configuration(TEST_UTIL.getConfiguration()));
+    List<Mutation> pending = new ArrayList<Mutation>();
+    for (int i = 0; i < 1000; i++) {
+      byte[] row = Bytes.toBytes("key" + Integer.toString(i));
+      Put put = new Put(row);
+      put.addColumn(family, c3, val);
+      put.addColumn(family, c4, val);
+      put.addColumn(family, c5, val);
+      put.addColumn(family, c6, val);
+      put.addColumn(family, c7, val);
+      put.addColumn(family, c8, val);
+      put.addColumn(family, c12, val);
+      put.addColumn(family, c13, val);
+      put.addColumn(family, c15, val);
+      pending.add(put);
+      Delete del = new Delete(row);
+      del.addColumns(family, c2);
+      del.addColumns(family, c9);
+      del.addColumns(family, c10);
+      del.addColumns(family, c14);
+      pending.add(del);
+    }
+    ht.batch(pending, new Object[pending.size()]);
+    TEST_UTIL.flush();
+    TEST_UTIL.compact(true);
+    for (int i = 20; i < 300; i++) {
+      byte[] row = Bytes.toBytes("key" + Integer.toString(i));
+      Put put = new Put(row);
+      put.addColumn(family, c3, val);
+      put.addColumn(family, c4, val);
+      put.addColumn(family, c5, val);
+      put.addColumn(family, c6, val);
+      put.addColumn(family, c7, val);
+      put.addColumn(family, c8, val);
+      put.addColumn(family, c12, val);
+      put.addColumn(family, c13, val);
+      put.addColumn(family, c15, val);
+      pending.add(put);
+      Delete del = new Delete(row);
+      del.addColumns(family, c2);
+      del.addColumns(family, c9);
+      del.addColumns(family, c10);
+      del.addColumns(family, c14);
+      pending.add(del);
+    }
+    ht.batch(pending, new Object[pending.size()]);
+    TEST_UTIL.flush();
+
+    Scan scan = new Scan();
+    scan.addColumn(family, c9);
+    scan.addColumn(family, c15);
+    SingleColumnValueFilter filter =
+        new SingleColumnValueFilter(family, c15, CompareFilter.CompareOp.EQUAL,
+            new BinaryComparator(c15));
+    scan.setFilter(filter);
+    //Trigger the scan for not existing row, so it will scan over all rows
+    for (Result result : ht.getScanner(scan)) {
+      result.advance();
+    }
+    ht.close();
+  }
+}
\ No newline at end of file