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 2008/11/19 03:11:41 UTC

svn commit: r718842 - in /hadoop/hbase/branches/0.19_on_hadoop_0.18: ./ src/java/org/apache/hadoop/hbase/filter/ src/test/org/apache/hadoop/hbase/filter/

Author: apurtell
Date: Tue Nov 18 18:11:41 2008
New Revision: 718842

URL: http://svn.apache.org/viewvc?rev=718842&view=rev
Log:
merge up to trunk (revision 718839)

Added:
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
Modified:
    hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt
    hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt?rev=718842&r1=718841&r2=718842&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/CHANGES.txt Tue Nov 18 18:11:41 2008
@@ -76,6 +76,7 @@
                TRUNK/0.19.0.
    HBASE-1003  If cell exceeds TTL but not VERSIONs, will not be removed during
                major compaction
+   HBASE-1005  Regex and string comparison operators for ColumnValueFilter
       
   IMPROVEMENTS
    HBASE-901   Add a limit to key length, check key and value length on client side

Modified: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java?rev=718842&r1=718841&r2=718842&view=diff
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java (original)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java Tue Nov 18 18:11:41 2008
@@ -53,11 +53,7 @@
     /** greater than or equal to */
     GREATER_OR_EQUAL,
     /** greater than */
-    GREATER,
-    // Below are more specific operators.
-    /** sub-string. Case insensitive. */
-    SUB_STRING;
-    
+    GREATER;
   }
 
   private byte[] columnName;
@@ -143,12 +139,6 @@
   }
 
   private boolean filterColumnValue(final byte [] data) {
-    // Special case for Substring operator
-    if (compareOp == CompareOp.SUB_STRING) {
-      return !Bytes.toString(data).toLowerCase().contains(
-          (Bytes.toString(value)).toLowerCase());
-    }
-
     int compareResult;
     if (comparator != null) {
       compareResult = comparator.compareTo(data);
@@ -179,10 +169,11 @@
   }
 
   public boolean filterRow(final SortedMap<byte[], Cell> columns) {
+    if (columns == null)
+      return false;
     if (filterIfColumnMissing) {
       return !columns.containsKey(columnName);
     } 
-
     // Otherwise we must do the filter here
     Cell colCell = columns.get(columnName);
       if (colCell == null) {

Added: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java?rev=718842&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java (added)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java Tue Nov 18 18:11:41 2008
@@ -0,0 +1,85 @@
+/**
+ * Copyright 2008 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.regex.Pattern;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This comparator is for use with ColumnValueFilter, for filtering based on
+ * the value of a given column. Use it to test if a given regular expression
+ * matches a cell value in the column.
+ * <p>
+ * Only EQUAL or NOT_EQUAL tests are valid with this comparator. 
+ * <p>
+ * For example:
+ * <p>
+ * <pre>
+ * ColumnValueFilter cvf =
+ *   new ColumnValueFilter("col",
+ *     ColumnValueFilter.CompareOp.EQUAL,
+ *     new RegexStringComparator(
+ *       // v4 IP address
+ *       "(((25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9]?)\\.){3,3}" +
+ *         "(25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9]?))(\\/[0-9]+)?" +
+ *         "|" +
+ *       // v6 IP address
+ *       "((([\\dA-Fa-f]{1,4}:){7}[\\dA-Fa-f]{1,4})(:([\\d]{1,3}.)" +
+ *         "{3}[\\d]{1,3})?)(\\/[0-9]+)?"));
+ * </pre>
+ */
+public class RegexStringComparator implements WritableByteArrayComparable {
+
+  private Pattern pattern;
+
+  /** Nullary constructor for Writable */
+  public RegexStringComparator() {
+  }
+
+  /**
+   * Constructor
+   * @param expr a valid regular expression
+   */
+  public RegexStringComparator(String expr) {
+    this.pattern = Pattern.compile(expr);
+  }
+
+  @Override
+  public int compareTo(byte[] value) {
+    // Use find() for subsequence match instead of matches() (full sequence
+    // match) to adhere to the principle of least surprise.
+    return pattern.matcher(Bytes.toString(value)).find() ? 1 : 0;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.pattern = Pattern.compile(in.readUTF());
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(pattern.toString());
+  }
+
+}

Added: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java?rev=718842&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java (added)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/java/org/apache/hadoop/hbase/filter/SubstringComparator.java Tue Nov 18 18:11:41 2008
@@ -0,0 +1,74 @@
+/**
+ * Copyright 2008 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 org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * This comparator is for use with ColumnValueFilter, for filtering based on
+ * the value of a given column. Use it to test if a given substring appears
+ * in a cell value in the column. The comparison is case insensitive.
+ * <p>
+ * Only EQUAL or NOT_EQUAL tests are valid with this comparator. 
+ * <p>
+ * For example:
+ * <p>
+ * <pre>
+ * ColumnValueFilter cvf =
+ *   new ColumnValueFilter("col", ColumnValueFilter.CompareOp.EQUAL,
+ *     new SubstringComparator("substr"));
+ * </pre>
+ */
+public class SubstringComparator implements WritableByteArrayComparable {
+
+  private String substr;
+
+  /** Nullary constructor for Writable */
+  public SubstringComparator() {
+  }
+
+  /**
+   * Constructor
+   * @param substr the substring
+   */
+  public SubstringComparator(String substr) {
+    this.substr = substr.toLowerCase();
+  }
+
+  @Override
+  public int compareTo(byte[] value) {
+    return Bytes.toString(value).toLowerCase().contains(substr) ? 1 : 0;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    substr = in.readUTF();
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeUTF(substr);
+  }
+
+}

Added: hadoop/hbase/branches/0.19_on_hadoop_0.18/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.19_on_hadoop_0.18/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java?rev=718842&view=auto
==============================================================================
--- hadoop/hbase/branches/0.19_on_hadoop_0.18/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java (added)
+++ hadoop/hbase/branches/0.19_on_hadoop_0.18/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java Tue Nov 18 18:11:41 2008
@@ -0,0 +1,143 @@
+/**
+ * Copyright 2007 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the stop row filter
+ */
+public class TestColumnValueFilter extends TestCase {
+
+  private static final byte[] ROW = Bytes.toBytes("test");
+  private static final byte[] COLUMN = Bytes.toBytes("test:foo");
+  private static final byte[] VAL_1 = Bytes.toBytes("a");
+  private static final byte[] VAL_2 = Bytes.toBytes("ab");
+  private static final byte[] VAL_3 = Bytes.toBytes("abc");
+  private static final byte[] VAL_4 = Bytes.toBytes("abcd");
+  private static final byte[] FULLSTRING_1 = 
+    Bytes.toBytes("The quick brown fox jumps over the lazy dog.");
+  private static final byte[] FULLSTRING_2 = 
+    Bytes.toBytes("The slow grey fox trips over the lazy dog.");
+  private static final String QUICK_SUBSTR = "quick";
+  private static final String QUICK_REGEX = "[q][u][i][c][k]";
+
+  private RowFilterInterface basicFilterNew() {
+    return new ColumnValueFilter(COLUMN,
+        ColumnValueFilter.CompareOp.GREATER_OR_EQUAL, VAL_2);
+  }
+
+  private RowFilterInterface substrFilterNew() {
+    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+      new SubstringComparator(QUICK_SUBSTR));
+  }
+
+  private RowFilterInterface regexFilterNew() {
+    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+      new RegexStringComparator(QUICK_REGEX));
+  }
+
+  private void basicFilterTests(RowFilterInterface filter)
+      throws Exception {
+    assertTrue("basicFilter1", filter.filterColumn(ROW, COLUMN, VAL_1));
+    assertFalse("basicFilter2", filter.filterColumn(ROW, COLUMN, VAL_2));
+    assertFalse("basicFilter3", filter.filterColumn(ROW, COLUMN, VAL_3));
+    assertFalse("basicFilter4", filter.filterColumn(ROW, COLUMN, VAL_4));
+    assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("basicFilterNotNull", filter.filterRow(null));
+  }
+
+  private void substrFilterTests(RowFilterInterface filter) 
+      throws Exception {
+    assertTrue("substrTrue", filter.filterColumn(ROW, COLUMN, FULLSTRING_1));
+    assertFalse("substrFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2));
+    assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("substrFilterNotNull", filter.filterRow(null));
+  }
+
+  private void regexFilterTests(RowFilterInterface filter) 
+      throws Exception {
+    assertTrue("regexTrue", filter.filterColumn(ROW, COLUMN, FULLSTRING_1));
+    assertFalse("regexFalse", filter.filterColumn(ROW, COLUMN, FULLSTRING_2));
+    assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("regexFilterNotNull", filter.filterRow(null));
+  }
+
+  private RowFilterInterface serializationTest(RowFilterInterface filter)
+      throws Exception {
+    // Decompose filter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    filter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+  
+    // Recompose filter.
+    DataInputStream in =
+      new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new ColumnValueFilter();
+    newFilter.readFields(in);
+  
+    return newFilter;
+  }
+
+  RowFilterInterface basicFilter;
+  RowFilterInterface substrFilter;
+  RowFilterInterface regexFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    basicFilter = basicFilterNew();
+    substrFilter = substrFilterNew();
+    regexFilter = regexFilterNew();
+  }
+
+  /**
+   * Tests identification of the stop row
+   * @throws Exception
+   */
+  public void testStop() throws Exception {
+    basicFilterTests(basicFilter);
+    substrFilterTests(substrFilter);
+    regexFilterTests(regexFilter);
+  }
+
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    RowFilterInterface newFilter = serializationTest(basicFilter);
+    basicFilterTests(newFilter);
+    newFilter = serializationTest(substrFilter);
+    substrFilterTests(newFilter);
+    newFilter = serializationTest(regexFilter);
+    regexFilterTests(newFilter);
+  }
+
+}