hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From te...@apache.org
Subject svn commit: r1368472 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/filter/ main/java/org/apache/hadoop/hbase/regionserver/ test/java/org/apache/hadoop/hbase/filter/
Date Thu, 02 Aug 2012 13:45:26 GMT
Author: tedyu
Date: Thu Aug  2 13:45:26 2012
New Revision: 1368472

URL: http://svn.apache.org/viewvc?rev=1368472&view=rev
Log:
HBASE-6429 Filter with filterRow() returning true is incompatible with scan with limit (Jie
Huang)


Added:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Thu
Aug  2 13:45:26 2012
@@ -143,7 +143,7 @@ public interface Filter extends Writable
   public void filterRow(List<KeyValue> kvs);
 
   /**
-   * @return True if this filter actively uses filterRow(List).
+   * @return True if this filter actively uses filterRow(List) or filterRow().
    * Primarily used to check for conflicts with scans(such as scans
    * that do not read a full row at a time)
    */

Added: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java?rev=1368472&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
(added)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
Thu Aug  2 13:45:26 2012
@@ -0,0 +1,115 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * This is a Filter wrapper class which is used in the server side. Some filter
+ * related hooks can be defined in this wrapper. The only way to create a
+ * FilterWrapper instance is passing a client side Filter instance through
+ * {@link org.apache.hadoop.hbase.client.Scan#getFilter()}.
+ * 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FilterWrapper implements Filter {
+  Filter filter = null;
+
+  public FilterWrapper( Filter filter ) {
+    if (null == filter) {
+      // ensure the filter instance is not null
+      throw new NullPointerException("Cannot create FilterWrapper with null Filter");
+    }
+    this.filter = filter;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    this.filter.write(out);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.filter.readFields(in);
+  }
+
+  @Override
+  public void reset() {
+    this.filter.reset();
+  }
+
+  @Override
+  public boolean filterAllRemaining() {
+    return this.filter.filterAllRemaining();
+  }
+
+  @Override
+  public boolean filterRow() {
+    return this.filter.filterRow();
+  }
+
+  @Override
+  public KeyValue getNextKeyHint(KeyValue currentKV) {
+    return this.filter.getNextKeyHint(currentKV);
+  }
+
+  @Override
+  public boolean filterRowKey(byte[] buffer, int offset, int length) {
+    return this.filter.filterRowKey(buffer, offset, length);
+  }
+
+  @Override
+  public ReturnCode filterKeyValue(KeyValue v) {
+    return this.filter.filterKeyValue(v);
+  }
+
+  @Override
+  public KeyValue transform(KeyValue v) {
+    return this.filter.transform(v);
+  }
+
+  @Override
+  public boolean hasFilterRow() {
+    return this.filter.hasFilterRow();
+  }
+
+  @Override
+  public void filterRow(List<KeyValue> kvs) {
+    //To fix HBASE-6429, 
+    //Filter with filterRow() returning true is incompatible with scan with limit
+    //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
+    //2. filterRow() is merged with filterRow(kvs),
+    //so that to make all those row related filtering stuff in the same function.
+    this.filter.filterRow(kvs);
+    if (!kvs.isEmpty() && this.filter.filterRow()) {
+      kvs.clear();
+    }
+  }
+
+}
\ No newline at end of file

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
Thu Aug  2 13:45:26 2012
@@ -77,6 +77,10 @@ public class PageFilter extends FilterBa
     this.rowsAccepted++;
     return this.rowsAccepted > this.pageSize;
   }
+  
+  public boolean hasFilterRow() {
+    return true;
+  }
 
   public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments)
{
     Preconditions.checkArgument(filterArguments.size() == 1,

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
Thu Aug  2 13:45:26 2012
@@ -89,6 +89,10 @@ public class RandomRowFilter extends Fil
   public boolean filterRow() {
     return filterOutRow;
   }
+  
+  public boolean hasFilterRow() {
+    return true;
+  }
 
   @Override
   public boolean filterRowKey(byte[] buffer, int offset, int length) {
@@ -119,4 +123,4 @@ public class RandomRowFilter extends Fil
   public void write(DataOutput out) throws IOException {
     out.writeFloat(chance);
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
Thu Aug  2 13:45:26 2012
@@ -202,6 +202,10 @@ public class SingleColumnValueFilter ext
     // If column not found, return true if we filter if missing, false if not
     return this.foundColumn? !this.matchedColumn: this.filterIfMissing;
   }
+  
+  public boolean hasFilterRow() {
+    return true;
+  }
 
   public void reset() {
     foundColumn = false;

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java
Thu Aug  2 13:45:26 2012
@@ -88,6 +88,10 @@ public class SkipFilter extends FilterBa
   public boolean filterRow() {
     return filterRow;
   }
+    
+  public boolean hasFilterRow() {
+    return true;
+  }
 
   public void write(DataOutput out) throws IOException {
     out.writeUTF(this.filter.getClass().getName());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java
Thu Aug  2 13:45:26 2012
@@ -89,6 +89,10 @@ public class WhileMatchFilter extends Fi
     changeFAR(filterRow);
     return filterRow;
   }
+  
+  public boolean hasFilterRow() {
+    return true;
+  }
 
   public void write(DataOutput out) throws IOException {
     out.writeUTF(this.filter.getClass().getName());

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1368472&r1=1368471&r2=1368472&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
Thu Aug  2 13:45:26 2012
@@ -101,6 +101,7 @@ import org.apache.hadoop.hbase.client.co
 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterWrapper;
 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
 import org.apache.hadoop.hbase.io.HeapSize;
@@ -3436,7 +3437,12 @@ public class HRegion implements HeapSize
       //DebugPrint.println("HRegionScanner.<init>");
 
       this.maxResultSize = scan.getMaxResultSize();
-      this.filter = scan.getFilter();
+      if (scan.hasFilter()) {
+        this.filter = new FilterWrapper(scan.getFilter());
+      } else {
+        this.filter = null;
+      }
+      
       this.batch = scan.getBatch();
       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
         this.stopRow = null;
@@ -3566,10 +3572,7 @@ public class HRegion implements HeapSize
           if (filter != null && filter.hasFilterRow()) {
             filter.filterRow(results);
           }
-          if (filter != null && filter.filterRow()) {
-            results.clear();
-          }
-
+          
           return false;
         } else if (filterRowKey(currentRow)) {
           nextRow(currentRow);
@@ -3580,7 +3583,7 @@ public class HRegion implements HeapSize
             if (limit > 0 && results.size() == limit) {
               if (this.filter != null && filter.hasFilterRow()) {
                 throw new IncompatibleFilterException(
-                  "Filter with filterRow(List<KeyValue>) incompatible with scan with
limit!");
+                  "Filter whose hasFilterRow() returns true is incompatible with scan with
limit!");
               }
               return true; // we are expecting more yes, but also limited to how many we
can return.
             }
@@ -3595,7 +3598,7 @@ public class HRegion implements HeapSize
             filter.filterRow(results);
           }
 
-          if (results.isEmpty() || filterRow()) {
+          if (results.isEmpty()) {
             // this seems like a redundant step - we already consumed the row
             // there're no left overs.
             // the reasons for calling this method are:

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java?rev=1368472&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
(added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWithScanLimits.java
Thu Aug  2 13:45:26 2012
@@ -0,0 +1,190 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+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.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.hadoop.hbase.MediumTests;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test if Filter is incompatible with scan-limits
+ */
+@Category(MediumTests.class)
+public class TestFilterWithScanLimits {
+  private static final Log LOG = LogFactory
+      .getLog(TestFilterWithScanLimits.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = null;
+  private static HBaseAdmin admin = null;
+  private static byte[] name = Bytes.toBytes("test");
+
+  @Test
+  public void testScanWithLimit() {
+    int kv_number = 0;
+    try {
+      Scan scan = new Scan();
+      // set batch number as 2, which means each Result should contain 2 KVs at
+      // most
+      scan.setBatch(2);
+      SingleColumnValueFilter filter = new SingleColumnValueFilter(
+          Bytes.toBytes("f1"), Bytes.toBytes("c5"),
+          CompareFilter.CompareOp.EQUAL, new SubstringComparator("2_c5"));
+
+      // add filter after batch defined
+      scan.setFilter(filter);
+      HTable table = new HTable(conf, name);
+      ResultScanner scanner = table.getScanner(scan);
+      // Expect to get following row
+      // row2 => <f1:c1, 2_c1>, <f1:c2, 2_c2>,
+      // row2 => <f1:c3, 2_c3>, <f1:c4, 2_c4>,
+      // row2 => <f1:c5, 2_c5>
+
+      for (Result result : scanner) {
+        for (KeyValue kv : result.list()) {
+          kv_number++;
+          LOG.debug(kv_number + ". kv: " + kv);
+        }
+      }
+
+      scanner.close();
+      table.close();
+    } catch (Exception e) {
+      // no correct result is expected
+      assertNotNull("No IncompatibleFilterException catched", e);
+    }
+    LOG.debug("check the fetched kv number");
+    assertEquals("We should not get result(s) returned.", 0, kv_number);
+  }
+
+  private static void prepareData() {
+    try {
+      HTable table = new HTable(TestFilterWithScanLimits.conf, name);
+      assertTrue("Fail to create the table", admin.tableExists(name));
+      List<Put> puts = new ArrayList<Put>();
+
+      // row1 => <f1:c1, 1_c1>, <f1:c2, 1_c2>, <f1:c3, 1_c3>, <f1:c4,1_c4>,
+      // <f1:c5, 1_c5>
+      // row2 => <f1:c1, 2_c1>, <f1:c2, 2_c2>, <f1:c3, 2_c3>, <f1:c4,2_c4>,
+      // <f1:c5, 2_c5>
+      for (int i = 1; i < 4; i++) {
+        Put put = new Put(Bytes.toBytes("row" + i));
+        for (int j = 1; j < 6; j++) {
+          put.add(Bytes.toBytes("f1"), Bytes.toBytes("c" + j),
+              Bytes.toBytes(i + "_c" + j));
+        }
+        puts.add(put);
+      }
+
+      table.put(puts);
+      table.close();
+    } catch (IOException e) {
+      assertNull("Exception found while putting data into table", e);
+    }
+  }
+
+  private static void createTable() {
+    assertNotNull("HBaseAdmin is not initialized successfully.", admin);
+    if (admin != null) {
+
+      HTableDescriptor desc = new HTableDescriptor(name);
+      HColumnDescriptor coldef = new HColumnDescriptor(Bytes.toBytes("f1"));
+      desc.addFamily(coldef);
+
+      try {
+        admin.createTable(desc);
+        assertTrue("Fail to create the table", admin.tableExists(name));
+      } catch (IOException e) {
+        assertNull("Exception found while creating table", e);
+      }
+
+    }
+  }
+
+  private static void deleteTable() {
+    if (admin != null) {
+      try {
+        admin.disableTable(name);
+        admin.deleteTable(name);
+      } catch (IOException e) {
+        assertNull("Exception found deleting the table", e);
+      }
+    }
+  }
+
+  private static void initialize(Configuration conf) {
+    TestFilterWithScanLimits.conf = HBaseConfiguration.create(conf);
+    TestFilterWithScanLimits.conf.setInt("hbase.client.retries.number", 1);
+    try {
+      admin = new HBaseAdmin(conf);
+    } catch (MasterNotRunningException e) {
+      assertNull("Master is not running", e);
+    } catch (ZooKeeperConnectionException e) {
+      assertNull("Cannot connect to Zookeeper", e);
+    }
+    createTable();
+    prepareData();
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration config = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniCluster(1);
+    initialize(TEST_UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    deleteTable();
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @org.junit.Rule
+  public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
+}
\ No newline at end of file

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java?rev=1368472&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
(added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterWrapper.java
Thu Aug  2 13:45:26 2012
@@ -0,0 +1,200 @@
+/*
+ * Copyright The Apache Software Foundation
+ *
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.ZooKeeperConnectionException;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+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.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.hbase.MediumTests;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test if the FilterWrapper retains the same semantics defined in the
+ * {@link org.apache.hadoop.hbase.filter.Filter}
+ */
+@Category(MediumTests.class)
+public class TestFilterWrapper {
+  private static final Log LOG = LogFactory.getLog(TestFilterWrapper.class);
+
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static Configuration conf = null;
+  private static HBaseAdmin admin = null;
+  private static byte[] name = Bytes.toBytes("test");
+
+  @Test
+  public void testFilterWrapper() {
+    int kv_number = 0;
+    int row_number = 0;
+    try {
+      Scan scan = new Scan();
+      List<Filter> fs = new ArrayList<Filter>();
+
+      DependentColumnFilter f1 = new DependentColumnFilter(Bytes.toBytes("f1"),
+          Bytes.toBytes("c5"), true, CompareFilter.CompareOp.EQUAL,
+          new SubstringComparator("c5"));
+      PageFilter f2 = new PageFilter(2);
+      fs.add(f1);
+      fs.add(f2);
+      FilterList filter = new FilterList(fs);
+
+      scan.setFilter(filter);
+      HTable table = new HTable(conf, name);
+      ResultScanner scanner = table.getScanner(scan);
+
+      // row2 (c1-c4) and row3(c1-c4) are returned
+      for (Result result : scanner) {
+        row_number++;
+        for (KeyValue kv : result.list()) {
+          LOG.debug(kv_number + ". kv: " + kv);
+          kv_number++;
+          assertEquals("Returned row is not correct", new String(kv.getRow()),
+              "row" + ( row_number + 1 ));
+        }
+      }
+
+      scanner.close();
+      table.close();
+    } catch (Exception e) {
+      // no correct result is expected
+      assertNull("Exception happens in scan", e);
+    }
+    LOG.debug("check the fetched kv number");
+    assertEquals("We should get 8 results returned.", 8, kv_number);
+    assertEquals("We should get 2 rows returned", 2, row_number);
+  }
+
+  private static void prepareData() {
+    try {
+      HTable table = new HTable(TestFilterWrapper.conf, name);
+      assertTrue("Fail to create the table", admin.tableExists(name));
+      List<Put> puts = new ArrayList<Put>();
+
+      // row1 => <f1:c1, 1_c1, ts=1>, <f1:c2, 1_c2, ts=2>, <f1:c3, 1_c3,ts=3>,
+      // <f1:c4,1_c4, ts=4>, <f1:c5, 1_c5, ts=5>
+      // row2 => <f1:c1, 2_c1, ts=2>, <f1,c2, 2_c2, ts=2>, <f1:c3, 2_c3,ts=2>,
+      // <f1:c4,2_c4, ts=2>, <f1:c5, 2_c5, ts=2>
+      // row3 => <f1:c1, 3_c1, ts=3>, <f1:c2, 3_c2, ts=3>, <f1:c3, 3_c3,ts=2>,
+      // <f1:c4,3_c4, ts=3>, <f1:c5, 3_c5, ts=3>
+      for (int i = 1; i < 4; i++) {
+        Put put = new Put(Bytes.toBytes("row" + i));
+        for (int j = 1; j < 6; j++) {
+          long timestamp = j;
+          if (i != 1)
+            timestamp = i;
+          put.add(Bytes.toBytes("f1"), Bytes.toBytes("c" + j), timestamp,
+              Bytes.toBytes(i + "_c" + j));
+        }
+        puts.add(put);
+      }
+
+      table.put(puts);
+      table.close();
+    } catch (IOException e) {
+      assertNull("Exception found while putting data into table", e);
+    }
+  }
+
+  private static void createTable() {
+    assertNotNull("HBaseAdmin is not initialized successfully.", admin);
+    if (admin != null) {
+
+      HTableDescriptor desc = new HTableDescriptor(name);
+      HColumnDescriptor coldef = new HColumnDescriptor(Bytes.toBytes("f1"));
+      desc.addFamily(coldef);
+
+      try {
+        admin.createTable(desc);
+        assertTrue("Fail to create the table", admin.tableExists(name));
+      } catch (IOException e) {
+        assertNull("Exception found while creating table", e);
+      }
+
+    }
+  }
+
+  private static void deleteTable() {
+    if (admin != null) {
+      try {
+        admin.disableTable(name);
+        admin.deleteTable(name);
+      } catch (IOException e) {
+        assertNull("Exception found deleting the table", e);
+      }
+    }
+  }
+
+  private static void initialize(Configuration conf) {
+    TestFilterWrapper.conf = HBaseConfiguration.create(conf);
+    TestFilterWrapper.conf.setInt("hbase.client.retries.number", 1);
+    try {
+      admin = new HBaseAdmin(conf);
+    } catch (MasterNotRunningException e) {
+      assertNull("Master is not running", e);
+    } catch (ZooKeeperConnectionException e) {
+      assertNull("Cannot connect to Zookeeper", e);
+    }
+    createTable();
+    prepareData();
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    Configuration config = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniCluster(1);
+    initialize(TEST_UTIL.getConfiguration());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    deleteTable();
+    TEST_UTIL.shutdownMiniCluster();
+  }
+
+  @org.junit.Rule
+  public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
+}



Mime
View raw message