You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by gx...@apache.org on 2018/09/10 13:56:47 UTC

[1/4] hbase git commit: HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter

Repository: hbase
Updated Branches:
  refs/heads/branch-2 2be9196ba -> 1a7dba44c
  refs/heads/branch-2.0 3807eb85c -> e3cf59e7c
  refs/heads/branch-2.1 6ab9997d1 -> 1c8c7e10f
  refs/heads/master b09dbb443 -> 2aae247e3


HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter


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

Branch: refs/heads/master
Commit: 2aae247e3f8f8a393b403a82593bdc3a1ba81193
Parents: b09dbb4
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Mon Sep 10 17:20:48 2018 +0800
Committer: Guangxu Cheng <gu...@gmail.com>
Committed: Mon Sep 10 21:32:46 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/QualifierFilter.java    |   7 +-
 .../TestQualifierFilterWithEmptyQualifier.java  | 165 +++++++++++++++++++
 2 files changed, 167 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2aae247e/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index 5d544a9..9d1d8c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -78,11 +78,8 @@ public class QualifierFilter extends CompareFilter {
 
   @Override
   public ReturnCode filterCell(final Cell c) {
-    int qualifierLength = c.getQualifierLength();
-    if (qualifierLength > 0) {
-      if (compareQualifier(getCompareOperator(), this.comparator, c)) {
-        return ReturnCode.SKIP;
-      }
+    if (compareQualifier(getCompareOperator(), this.comparator, c)) {
+      return ReturnCode.SKIP;
     }
     return ReturnCode.INCLUDE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2aae247e/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
new file mode 100644
index 0000000..c1a3da6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
@@ -0,0 +1,165 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+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.CellComparator;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test qualifierFilter with empty qualifier column
+ */
+@Category({FilterTests.class, SmallTests.class})
+public class TestQualifierFilterWithEmptyQualifier {
+
+  private final static Logger LOG
+      = LoggerFactory.getLogger(TestQualifierFilterWithEmptyQualifier.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestQualifierFilterWithEmptyQualifier.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private HRegion region;
+
+  @Rule
+  public TestName name = new TestName();
+
+  private static final byte[][] ROWS =
+    { Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
+        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") };
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte[][] QUALIFIERS = {HConstants.EMPTY_BYTE_ARRAY,
+      Bytes.toBytes("testQualifier")};
+  private static final byte[] VALUE = Bytes.toBytes("testValueOne");
+  private long numRows = (long) ROWS.length;
+
+  @Before
+  public void setUp() throws Exception {
+    TableDescriptor htd = TableDescriptorBuilder
+        .newBuilder(TableName.valueOf("TestQualifierFilter"))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    this.region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+
+    // Insert data
+    for (byte[] ROW : ROWS) {
+      Put p = new Put(ROW);
+      p.setDurability(Durability.SKIP_WAL);
+      for (byte[] QUALIFIER : QUALIFIERS) {
+        p.addColumn(FAMILY, QUALIFIER, VALUE);
+      }
+      this.region.put(p);
+    }
+
+    // Flush
+    this.region.flush(true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  @Test
+  public void testQualifierFilterWithEmptyColumn() throws IOException {
+    long colsPerRow = 2;
+    long expectedKeys = colsPerRow / 2;
+    Filter f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    Scan s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[1]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.GREATER,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow;
+    f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+  }
+
+  private void verifyScanNoEarlyOut(Scan s, long expectedRows,
+      long expectedKeys)
+      throws IOException {
+    InternalScanner scanner = this.region.getScanner(s);
+    List<Cell> results = new ArrayList<>();
+    int i = 0;
+    for (boolean done = true; done; i++) {
+      done = scanner.next(results);
+      Arrays.sort(results.toArray(new Cell[results.size()]),
+          CellComparator.getInstance());
+      LOG.info("counter=" + i + ", " + results);
+      if(results.isEmpty()) {
+        break;
+      }
+      assertTrue("Scanned too many rows! Only expected " + expectedRows +
+          " total but already scanned " + (i+1), expectedRows > i);
+      assertEquals("Expected " + expectedKeys + " keys per row but " +
+          "returned " + results.size(), expectedKeys, results.size());
+      results.clear();
+    }
+    assertEquals("Expected " + expectedRows + " rows but scanned " + i +
+        " rows", expectedRows, i);
+  }
+}
\ No newline at end of file


[2/4] hbase git commit: HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter

Posted by gx...@apache.org.
HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter


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

Branch: refs/heads/branch-2.1
Commit: 1c8c7e10f8bb0298f031d3e03029f62098f66997
Parents: 6ab9997
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Mon Sep 10 17:20:48 2018 +0800
Committer: Guangxu Cheng <gu...@gmail.com>
Committed: Mon Sep 10 21:40:57 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/QualifierFilter.java    |   7 +-
 .../TestQualifierFilterWithEmptyQualifier.java  | 165 +++++++++++++++++++
 2 files changed, 167 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1c8c7e10/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index b38c010..941575a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -78,11 +78,8 @@ public class QualifierFilter extends CompareFilter {
 
   @Override
   public ReturnCode filterCell(final Cell c) {
-    int qualifierLength = c.getQualifierLength();
-    if (qualifierLength > 0) {
-      if (compareQualifier(getCompareOperator(), this.comparator, c)) {
-        return ReturnCode.SKIP;
-      }
+    if (compareQualifier(getCompareOperator(), this.comparator, c)) {
+      return ReturnCode.SKIP;
     }
     return ReturnCode.INCLUDE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1c8c7e10/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
new file mode 100644
index 0000000..c1a3da6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
@@ -0,0 +1,165 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+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.CellComparator;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test qualifierFilter with empty qualifier column
+ */
+@Category({FilterTests.class, SmallTests.class})
+public class TestQualifierFilterWithEmptyQualifier {
+
+  private final static Logger LOG
+      = LoggerFactory.getLogger(TestQualifierFilterWithEmptyQualifier.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestQualifierFilterWithEmptyQualifier.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private HRegion region;
+
+  @Rule
+  public TestName name = new TestName();
+
+  private static final byte[][] ROWS =
+    { Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
+        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") };
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte[][] QUALIFIERS = {HConstants.EMPTY_BYTE_ARRAY,
+      Bytes.toBytes("testQualifier")};
+  private static final byte[] VALUE = Bytes.toBytes("testValueOne");
+  private long numRows = (long) ROWS.length;
+
+  @Before
+  public void setUp() throws Exception {
+    TableDescriptor htd = TableDescriptorBuilder
+        .newBuilder(TableName.valueOf("TestQualifierFilter"))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    this.region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+
+    // Insert data
+    for (byte[] ROW : ROWS) {
+      Put p = new Put(ROW);
+      p.setDurability(Durability.SKIP_WAL);
+      for (byte[] QUALIFIER : QUALIFIERS) {
+        p.addColumn(FAMILY, QUALIFIER, VALUE);
+      }
+      this.region.put(p);
+    }
+
+    // Flush
+    this.region.flush(true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  @Test
+  public void testQualifierFilterWithEmptyColumn() throws IOException {
+    long colsPerRow = 2;
+    long expectedKeys = colsPerRow / 2;
+    Filter f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    Scan s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[1]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.GREATER,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow;
+    f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+  }
+
+  private void verifyScanNoEarlyOut(Scan s, long expectedRows,
+      long expectedKeys)
+      throws IOException {
+    InternalScanner scanner = this.region.getScanner(s);
+    List<Cell> results = new ArrayList<>();
+    int i = 0;
+    for (boolean done = true; done; i++) {
+      done = scanner.next(results);
+      Arrays.sort(results.toArray(new Cell[results.size()]),
+          CellComparator.getInstance());
+      LOG.info("counter=" + i + ", " + results);
+      if(results.isEmpty()) {
+        break;
+      }
+      assertTrue("Scanned too many rows! Only expected " + expectedRows +
+          " total but already scanned " + (i+1), expectedRows > i);
+      assertEquals("Expected " + expectedKeys + " keys per row but " +
+          "returned " + results.size(), expectedKeys, results.size());
+      results.clear();
+    }
+    assertEquals("Expected " + expectedRows + " rows but scanned " + i +
+        " rows", expectedRows, i);
+  }
+}
\ No newline at end of file


[4/4] hbase git commit: HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter

Posted by gx...@apache.org.
HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter


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

Branch: refs/heads/branch-2
Commit: 1a7dba44cecc2f8bbdca984e24ab8980ade2d77c
Parents: 2be9196
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Mon Sep 10 17:20:48 2018 +0800
Committer: Guangxu Cheng <gu...@gmail.com>
Committed: Mon Sep 10 21:51:48 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/QualifierFilter.java    |   7 +-
 .../TestQualifierFilterWithEmptyQualifier.java  | 165 +++++++++++++++++++
 2 files changed, 167 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/1a7dba44/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index 5d544a9..9d1d8c7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -78,11 +78,8 @@ public class QualifierFilter extends CompareFilter {
 
   @Override
   public ReturnCode filterCell(final Cell c) {
-    int qualifierLength = c.getQualifierLength();
-    if (qualifierLength > 0) {
-      if (compareQualifier(getCompareOperator(), this.comparator, c)) {
-        return ReturnCode.SKIP;
-      }
+    if (compareQualifier(getCompareOperator(), this.comparator, c)) {
+      return ReturnCode.SKIP;
     }
     return ReturnCode.INCLUDE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/1a7dba44/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
new file mode 100644
index 0000000..c1a3da6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
@@ -0,0 +1,165 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+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.CellComparator;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test qualifierFilter with empty qualifier column
+ */
+@Category({FilterTests.class, SmallTests.class})
+public class TestQualifierFilterWithEmptyQualifier {
+
+  private final static Logger LOG
+      = LoggerFactory.getLogger(TestQualifierFilterWithEmptyQualifier.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestQualifierFilterWithEmptyQualifier.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private HRegion region;
+
+  @Rule
+  public TestName name = new TestName();
+
+  private static final byte[][] ROWS =
+    { Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
+        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") };
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte[][] QUALIFIERS = {HConstants.EMPTY_BYTE_ARRAY,
+      Bytes.toBytes("testQualifier")};
+  private static final byte[] VALUE = Bytes.toBytes("testValueOne");
+  private long numRows = (long) ROWS.length;
+
+  @Before
+  public void setUp() throws Exception {
+    TableDescriptor htd = TableDescriptorBuilder
+        .newBuilder(TableName.valueOf("TestQualifierFilter"))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    this.region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+
+    // Insert data
+    for (byte[] ROW : ROWS) {
+      Put p = new Put(ROW);
+      p.setDurability(Durability.SKIP_WAL);
+      for (byte[] QUALIFIER : QUALIFIERS) {
+        p.addColumn(FAMILY, QUALIFIER, VALUE);
+      }
+      this.region.put(p);
+    }
+
+    // Flush
+    this.region.flush(true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  @Test
+  public void testQualifierFilterWithEmptyColumn() throws IOException {
+    long colsPerRow = 2;
+    long expectedKeys = colsPerRow / 2;
+    Filter f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    Scan s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[1]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.GREATER,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow;
+    f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+  }
+
+  private void verifyScanNoEarlyOut(Scan s, long expectedRows,
+      long expectedKeys)
+      throws IOException {
+    InternalScanner scanner = this.region.getScanner(s);
+    List<Cell> results = new ArrayList<>();
+    int i = 0;
+    for (boolean done = true; done; i++) {
+      done = scanner.next(results);
+      Arrays.sort(results.toArray(new Cell[results.size()]),
+          CellComparator.getInstance());
+      LOG.info("counter=" + i + ", " + results);
+      if(results.isEmpty()) {
+        break;
+      }
+      assertTrue("Scanned too many rows! Only expected " + expectedRows +
+          " total but already scanned " + (i+1), expectedRows > i);
+      assertEquals("Expected " + expectedKeys + " keys per row but " +
+          "returned " + results.size(), expectedKeys, results.size());
+      results.clear();
+    }
+    assertEquals("Expected " + expectedRows + " rows but scanned " + i +
+        " rows", expectedRows, i);
+  }
+}
\ No newline at end of file


[3/4] hbase git commit: HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter

Posted by gx...@apache.org.
HBASE-21158 Empty qualifier cell is always returned when using QualifierFilter


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

Branch: refs/heads/branch-2.0
Commit: e3cf59e7c38c04f6ba9161582940baa2ae295843
Parents: 3807eb8
Author: Guangxu Cheng <gu...@gmail.com>
Authored: Mon Sep 10 17:20:48 2018 +0800
Committer: Guangxu Cheng <gu...@gmail.com>
Committed: Mon Sep 10 21:48:47 2018 +0800

----------------------------------------------------------------------
 .../hadoop/hbase/filter/QualifierFilter.java    |   7 +-
 .../TestQualifierFilterWithEmptyQualifier.java  | 165 +++++++++++++++++++
 2 files changed, 167 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3cf59e7/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
index b38c010..941575a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
@@ -78,11 +78,8 @@ public class QualifierFilter extends CompareFilter {
 
   @Override
   public ReturnCode filterCell(final Cell c) {
-    int qualifierLength = c.getQualifierLength();
-    if (qualifierLength > 0) {
-      if (compareQualifier(getCompareOperator(), this.comparator, c)) {
-        return ReturnCode.SKIP;
-      }
+    if (compareQualifier(getCompareOperator(), this.comparator, c)) {
+      return ReturnCode.SKIP;
     }
     return ReturnCode.INCLUDE;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e3cf59e7/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
new file mode 100644
index 0000000..c1a3da6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestQualifierFilterWithEmptyQualifier.java
@@ -0,0 +1,165 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+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.CellComparator;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.testclassification.FilterTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test qualifierFilter with empty qualifier column
+ */
+@Category({FilterTests.class, SmallTests.class})
+public class TestQualifierFilterWithEmptyQualifier {
+
+  private final static Logger LOG
+      = LoggerFactory.getLogger(TestQualifierFilterWithEmptyQualifier.class);
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestQualifierFilterWithEmptyQualifier.class);
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private HRegion region;
+
+  @Rule
+  public TestName name = new TestName();
+
+  private static final byte[][] ROWS =
+    { Bytes.toBytes("testRowOne-0"), Bytes.toBytes("testRowOne-1"),
+        Bytes.toBytes("testRowOne-2"), Bytes.toBytes("testRowOne-3") };
+  private static final byte[] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte[][] QUALIFIERS = {HConstants.EMPTY_BYTE_ARRAY,
+      Bytes.toBytes("testQualifier")};
+  private static final byte[] VALUE = Bytes.toBytes("testValueOne");
+  private long numRows = (long) ROWS.length;
+
+  @Before
+  public void setUp() throws Exception {
+    TableDescriptor htd = TableDescriptorBuilder
+        .newBuilder(TableName.valueOf("TestQualifierFilter"))
+        .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
+    this.region = HBaseTestingUtility
+        .createRegionAndWAL(info, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(), htd);
+
+    // Insert data
+    for (byte[] ROW : ROWS) {
+      Put p = new Put(ROW);
+      p.setDurability(Durability.SKIP_WAL);
+      for (byte[] QUALIFIER : QUALIFIERS) {
+        p.addColumn(FAMILY, QUALIFIER, VALUE);
+      }
+      this.region.put(p);
+    }
+
+    // Flush
+    this.region.flush(true);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    HBaseTestingUtility.closeRegionAndWAL(region);
+  }
+
+  @Test
+  public void testQualifierFilterWithEmptyColumn() throws IOException {
+    long colsPerRow = 2;
+    long expectedKeys = colsPerRow / 2;
+    Filter f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    Scan s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.EQUAL,
+        new BinaryComparator(QUALIFIERS[1]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow / 2;
+    f = new QualifierFilter(CompareOperator.GREATER,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+
+    expectedKeys = colsPerRow;
+    f = new QualifierFilter(CompareOperator.GREATER_OR_EQUAL,
+        new BinaryComparator(QUALIFIERS[0]));
+    s = new Scan();
+    s.setFilter(f);
+    verifyScanNoEarlyOut(s, this.numRows, expectedKeys);
+  }
+
+  private void verifyScanNoEarlyOut(Scan s, long expectedRows,
+      long expectedKeys)
+      throws IOException {
+    InternalScanner scanner = this.region.getScanner(s);
+    List<Cell> results = new ArrayList<>();
+    int i = 0;
+    for (boolean done = true; done; i++) {
+      done = scanner.next(results);
+      Arrays.sort(results.toArray(new Cell[results.size()]),
+          CellComparator.getInstance());
+      LOG.info("counter=" + i + ", " + results);
+      if(results.isEmpty()) {
+        break;
+      }
+      assertTrue("Scanned too many rows! Only expected " + expectedRows +
+          " total but already scanned " + (i+1), expectedRows > i);
+      assertEquals("Expected " + expectedKeys + " keys per row but " +
+          "returned " + results.size(), expectedKeys, results.size());
+      results.clear();
+    }
+    assertEquals("Expected " + expectedRows + " rows but scanned " + i +
+        " rows", expectedRows, i);
+  }
+}
\ No newline at end of file