hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r659164 - in /hadoop/hbase/trunk: CHANGES.txt src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
Date Thu, 22 May 2008 17:17:08 GMT
Author: stack
Date: Thu May 22 10:17:07 2008
New Revision: 659164

URL: http://svn.apache.org/viewvc?rev=659164&view=rev
Log:
HBASE-583 RangeRowFilter, allow to choose rows based on a (lexicographic) comparison to column's
values

Added:
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=659164&r1=659163&r2=659164&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu May 22 10:17:07 2008
@@ -45,6 +45,9 @@
                compaction (Jean-Daniel Cryans via Stack)
    HBASE-504   Allow HMsg's carry a payload: e.g. exception that happened over
                on the remote side.
+   HBASE-583   RangeRowFilter/ColumnValueFilter to allow choice of rows based on
+               a (lexicographic) comparison to column's values
+               (Clint Morgan via Stack)
 
 Release 0.1.2 - 05/13/2008
    

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java?rev=659164&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java Thu
May 22 10:17:07 2008
@@ -0,0 +1,189 @@
+/**
+ * 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.Arrays;
+import java.util.SortedMap;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.ObjectWritable;
+
+/**
+ * This filter is used to filter based on the value of a given column. It takes
+ * an operator (equal, greater, not equal, etc) and either a byte [] value or a
+ * byte [] comparator. If we have a byte [] value then we just do a
+ * lexicographic compare. If this is not sufficient (eg you want to deserialize
+ * a long and then compare it to a fixed long value, then you can pass in your
+ * own comparator instead.
+ */
+public class ColumnValueFilter implements RowFilterInterface {
+
+  /** Comparison operator. */
+  public enum CompareOp {
+    LESS, LESS_OR_EQUAL, EQUAL, NOT_EQUAL, GREATER_OR_EQUAL, GREATER;
+  }
+
+  private byte[] columnName;
+  private CompareOp compareOp;
+  private byte[] value;
+  private WritableByteArrayComparable comparator;
+
+  ColumnValueFilter() {
+    // for Writable
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param columnName name of column
+   * @param compareOp operator
+   * @param value value to compare column values against
+   */
+  public ColumnValueFilter(final byte[] columnName, final CompareOp compareOp,
+      final byte[] value) {
+    this.columnName = columnName;
+    this.compareOp = compareOp;
+    this.value = value;
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param columnName name of column
+   * @param compareOp operator
+   * @param comparator Comparator to use.
+   */
+  public ColumnValueFilter(final byte[] columnName, final CompareOp compareOp,
+      final WritableByteArrayComparable comparator) {
+    this.columnName = columnName;
+    this.compareOp = compareOp;
+    this.comparator = comparator;
+  }
+
+  /** {@inheritDoc} */
+  public boolean filterRowKey(final byte[] rowKey) {
+    return false;
+  }
+
+  /** {@inheritDoc} */
+  public boolean filterColumn(final byte[] rowKey, final byte[] colKey,
+      final byte[] data) {
+    if (!Arrays.equals(colKey, columnName)) {
+      return false;
+    }
+
+    int compareResult;
+    if (comparator != null) {
+      compareResult = comparator.compareTo(data);
+    } else {
+      compareResult = compare(value, data);
+    }
+
+    switch (compareOp) {
+    case LESS:
+      return compareResult <= 0;
+    case LESS_OR_EQUAL:
+      return compareResult < 0;
+    case EQUAL:
+      return compareResult != 0;
+    case NOT_EQUAL:
+      return compareResult == 0;
+    case GREATER_OR_EQUAL:
+      return compareResult > 0;
+    case GREATER:
+      return compareResult >= 0;
+    default:
+      throw new RuntimeException("Unknown Compare op " + compareOp.name());
+    }
+  }
+
+  /** {@inheritDoc} */
+  public boolean filterAllRemaining() {
+    return false;
+  }
+
+  /** {@inheritDoc} */
+  public boolean filterRow(final SortedMap<byte[], byte[]> columns) {
+    // Don't let rows through if they don't have the column we are checking
+    return !columns.containsKey(columnName);
+  }
+
+  private int compare(final byte[] b1, final byte[] b2) {
+    int len = Math.min(b1.length, b2.length);
+
+    for (int i = 0; i < len; i++) {
+      if (b1[i] != b2[i]) {
+        return b1[i] - b2[i];
+      }
+    }
+    return b1.length - b2.length;
+  }
+
+  /** {@inheritDoc} */
+  public boolean processAlways() {
+    return false;
+  }
+
+  /** {@inheritDoc} */
+  public void reset() {
+    // Nothing.
+  }
+
+  public void rowProcessed(final boolean filtered, final byte[] key) {
+    // Nothing
+  }
+
+  /** {@inheritDoc} */
+  public void validate(final byte[][] columns) {
+    // Nothing
+  }
+
+  /** {@inheritDoc} */
+  public void readFields(final DataInput in) throws IOException {
+    int valueLen = in.readInt();
+    if (valueLen > 0) {
+      value = new byte[valueLen];
+      in.readFully(value);
+    }
+    columnName = Bytes.readByteArray(in);
+    compareOp = CompareOp.valueOf(in.readUTF());
+    comparator = (WritableByteArrayComparable) ObjectWritable.readObject(in,
+        new HBaseConfiguration());
+  }
+
+  /** {@inheritDoc} */
+  public void write(final DataOutput out) throws IOException {
+    if (value == null) {
+      out.writeInt(0);
+    } else {
+      out.writeInt(value.length);
+      out.write(value);
+    }
+    Bytes.writeByteArray(out, columnName);
+    out.writeUTF(compareOp.name());
+    ObjectWritable.writeObject(out, comparator,
+        WritableByteArrayComparable.class, new HBaseConfiguration());
+  }
+
+}

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=659164&r1=659163&r2=659164&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Thu May
22 10:17:07 2008
@@ -36,8 +36,11 @@
 
 /**
  * Implementation of RowFilterInterface that can filter by rowkey regular
- * expression and/or individual column values (equals comparison only).
- * Multiple column filters imply an implicit conjunction of filter criteria.
+ * expression and/or individual column values (equals comparison only). Multiple
+ * column filters imply an implicit conjunction of filter criteria.
+ * 
+ * Note that column value filtering in this interface has been replaced by
+ * {@link ColumnValueFilter}.
  */
 public class RegExpRowFilter implements RowFilterInterface {
 
@@ -67,11 +70,13 @@
   }
 
   /**
+   * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
    * Constructor that takes a row key regular expression to filter on.
    * 
    * @param rowKeyRegExp
    * @param columnFilter
    */
+  @Deprecated
   public RegExpRowFilter(final String rowKeyRegExp,
       final Map<byte [], byte[]> columnFilter) {
     this.rowKeyRegExp = rowKeyRegExp;
@@ -90,6 +95,7 @@
   }
   
   /**
+   * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
    * Specify a value that must be matched for the given column.
    * 
    * @param colKey
@@ -97,6 +103,7 @@
    * @param value
    *          the value that must equal the stored value.
    */
+  @Deprecated
   public void setColumnFilter(final byte [] colKey, final byte[] value) {
     if (value == null) {
       nullColumns.add(colKey);
@@ -106,11 +113,13 @@
   }
 
   /**
+   * @deprecated Column filtering has been replaced by {@link ColumnValueFilter}
    * Set column filters for a number of columns.
    * 
    * @param columnFilter
    *          Map of columns with value criteria.
    */
+  @Deprecated
   public void setColumnFilters(final Map<byte [], byte[]> columnFilter) {
     if (null == columnFilter) {
       nullColumns.clear();

Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java?rev=659164&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
(added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/WritableByteArrayComparable.java
Thu May 22 10:17:07 2008
@@ -0,0 +1,28 @@
+/**
+ * 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 org.apache.hadoop.io.Writable;
+
+/** Interface for both Comparable<byte []> and Writable. */
+public interface WritableByteArrayComparable extends Writable,
+    Comparable<byte[]> {
+  // Not methods, just tie the two interfaces together.
+}



Mime
View raw message