You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 19:43:29 UTC
svn commit: r1181945 [1/2] - in /hbase/branches/0.89/src:
main/java/org/apache/hadoop/hbase/filter/ main/ruby/ main/ruby/hbase/
main/ruby/shell/commands/ test/java/org/apache/hadoop/hbase/filter/
Author: nspiegelberg
Date: Tue Oct 11 17:43:28 2011
New Revision: 1181945
URL: http://svn.apache.org/viewvc?rev=1181945&view=rev
Log:
Backporting Filter Language diff from hbase-90.
Also includes support of the Filter Language from the HBase shell
Summary:
Committed https://phabricator.fb.com/D277637 and
https://phabricator.fb.com/D300515 to hbase-90.
This diff is pretty similar.
Test Plan:
TestParseFilter.java
Tested manually from the shell.
hbase(main):003:0> scan 'conf', {FILTER => "KeyOnlyFilter()"}
ROW COLUMN+CELL
realtime domain_threshold_imp's column=conf:number, timestamp=1311055899185,
value=
realtime domain_threshold_imp's column=conf:sameplesize,
timestamp=1310962821624, value=
realtime domain_threshold_imps column=conf:sameplesize,
timestamp=1310962794886, value=
realtime_domain_acts column=conf:blacklist, timestamp=1310667339990, value=
realtime_domain_acts column=conf:daily_thresholds, timestamp=1310667461494,
value=
hbase(main):002:0> scan 'conf', {FILTER =>
org.apache.hadoop.hbase.filter.KeyOnlyFilter.new()}
ROW COLUMN+CELL
realtime domain_threshold_imp's column=conf:number, timestamp=1311055899185,
value=
realtime domain_threshold_imp's column=conf:sameplesize,
timestamp=1310962821624, value=
realtime domain_threshold_imps column=conf:sameplesize,
timestamp=1310962794886, value=
realtime_domain_acts column=conf:blacklist, timestamp=1310667339990, value=
realtime_domain_acts column=conf:daily_thresholds, timestamp=1310667461494,
value=
hbase(main):005:0> scan 'conf', {FILTER => "(FirstKeyOnlyFilter() AND
ValueFilter(>=, 'binary:ghi')) OR TimestampsFilter(1311109736514)"}
ROW COLUMN+CELL
realtime domain_threshold_imp\'s column=conf:sameplesize,
timestamp=1310970483721, value=nonononono
realtime domain_threshold_imps column=conf:sameplesize,
timestamp=1310962794886, value=nonononono
realtime_domain_acts column=conf:blacklist, timestamp=1310667339990,
value=hello
row1 column=conf:number, timestamp=1311109736514, value=short_row
Reviewed By: liyintang
Reviewers: pkhemani, jgray, liyintang, kannan, kranganathan, nspiegelberg
CC: hbase@lists, liyintang
Differential Revision: 301887
Added:
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
hbase/branches/0.89/src/main/ruby/shell/commands/locate_regionservers.rb
hbase/branches/0.89/src/main/ruby/shell/commands/show_filters.rb
hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
Modified:
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
hbase/branches/0.89/src/main/ruby/hbase.rb
hbase/branches/0.89/src/main/ruby/hbase/table.rb
hbase/branches/0.89/src/main/ruby/shell.rb
hbase/branches/0.89/src/main/ruby/shell/commands/scan.rb
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java Tue Oct 11 17:43:28 2011
@@ -25,6 +25,9 @@ import org.apache.hadoop.hbase.KeyValue;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* Simple filter that returns first N columns on row only.
@@ -68,6 +71,13 @@ public class ColumnCountGetFilter extend
this.count = 0;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 1,
+ "Expected 1 but got: %s", filterArguments.size());
+ int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
+ return new ColumnCountGetFilter(limit);
+ }
+
@Override
public void readFields(DataInput in) throws IOException {
this.limit = in.readInt();
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java Tue Oct 11 17:43:28 2011
@@ -23,8 +23,10 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
+import java.util.ArrayList;
import org.apache.hadoop.hbase.KeyValue;
+import com.google.common.base.Preconditions;
/**
* A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
@@ -47,6 +49,8 @@ public class ColumnPaginationFilter exte
public ColumnPaginationFilter(final int limit, final int offset)
{
+ Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
+ Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset);
this.limit = limit;
this.offset = offset;
}
@@ -70,6 +74,22 @@ public class ColumnPaginationFilter exte
this.count = 0;
}
+ public int getLimit() {
+ return this.limit;
+ }
+
+ public int getOffset() {
+ return this.offset;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 2,
+ "Expected 2 but got: %s", filterArguments.size());
+ int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
+ int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
+ return new ColumnPaginationFilter(limit, offset);
+ }
+
public void readFields(DataInput in) throws IOException
{
this.limit = in.readInt();
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java Tue Oct 11 17:43:28 2011
@@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Byte
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* This filter is used for selecting only those keys with columns that matches
@@ -43,6 +46,13 @@ public class ColumnPrefixFilter extends
this.prefix = prefix;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 1,
+ "Expected 1 but got: %s", filterArguments.size());
+ byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ return new ColumnPrefixFilter(columnPrefix);
+ }
+
public byte[] getPrefix() {
return prefix;
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java Tue Oct 11 17:43:28 2011
@@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Byte
import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* This filter is used for selecting only those keys with columns that are
@@ -129,6 +132,22 @@ public class ColumnRangeFilter extends F
return ReturnCode.NEXT_ROW;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 4,
+ "Expected 4 but got: %s", filterArguments.size());
+ byte [] minColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ boolean minColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(1));
+ byte [] maxColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(2));
+ boolean maxColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(3));
+
+ if (minColumn.length == 0)
+ minColumn = null;
+ if (maxColumn.length == 0)
+ maxColumn = null;
+ return new ColumnRangeFilter(minColumn, minColumnInclusive,
+ maxColumn, maxColumnInclusive);
+ }
+
@Override
public void write(DataOutput out) throws IOException {
// need to write out a flag for null value separately. Otherwise,
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java Tue Oct 11 17:43:28 2011
@@ -27,6 +27,9 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* This is a generic filter to be used to filter by comparison. It takes an
@@ -125,6 +128,27 @@ public abstract class CompareFilter exte
}
}
+ public static ArrayList extractArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 2,
+ "Expected 2 but got: %s", filterArguments.size());
+ CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(0));
+ WritableByteArrayComparable comparator = ParseFilter.createComparator(
+ ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)));
+
+ if (comparator instanceof RegexStringComparator ||
+ comparator instanceof SubstringComparator) {
+ if (compareOp != CompareOp.EQUAL &&
+ compareOp != CompareOp.NOT_EQUAL) {
+ throw new IllegalArgumentException ("A regexstring comparator and substring comparator" +
+ " can only be used with EQUAL and NOT_EQUAL");
+ }
+ }
+ ArrayList arguments = new ArrayList();
+ arguments.add(compareOp);
+ arguments.add(comparator);
+ return arguments;
+ }
+
public void readFields(DataInput in) throws IOException {
compareOp = CompareOp.valueOf(in.readUTF());
comparator = (WritableByteArrayComparable)
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Tue Oct 11 17:43:28 2011
@@ -25,17 +25,19 @@ import java.io.IOException;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
+import java.util.ArrayList;
import java.util.Set;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
+import com.google.common.base.Preconditions;
/**
* A filter for adding inter-column timestamp matching
* Only cells with a correspondingly timestamped entry in
* the target column will be retained
- * Not compatible with Scan.setBatch as operations need
- * full rows for correct filtering
+ * Not compatible with Scan.setBatch as operations need
+ * full rows for correct filtering
*/
public class DependentColumnFilter extends CompareFilter {
@@ -44,62 +46,62 @@ public class DependentColumnFilter exten
protected boolean dropDependentColumn;
protected Set<Long> stampSet = new HashSet<Long>();
-
+
/**
* Should only be used for writable
*/
public DependentColumnFilter() {
}
-
+
/**
* Build a dependent column filter with value checking
* dependent column varies will be compared using the supplied
* compareOp and comparator, for usage of which
* refer to {@link CompareFilter}
- *
+ *
* @param family dependent column family
* @param qualifier dependent column qualifier
* @param dropDependentColumn whether the column should be discarded after
- * @param valueCompareOp comparison op
+ * @param valueCompareOp comparison op
* @param valueComparator comparator
*/
public DependentColumnFilter(final byte [] family, final byte[] qualifier,
- final boolean dropDependentColumn, final CompareOp valueCompareOp,
- final WritableByteArrayComparable valueComparator) {
- // set up the comparator
+ final boolean dropDependentColumn, final CompareOp valueCompareOp,
+ final WritableByteArrayComparable valueComparator) {
+ // set up the comparator
super(valueCompareOp, valueComparator);
this.columnFamily = family;
this.columnQualifier = qualifier;
this.dropDependentColumn = dropDependentColumn;
}
-
+
/**
* Constructor for DependentColumn filter.
- * Keyvalues where a keyvalue from target column
- * with the same timestamp do not exist will be dropped.
- *
+ * Keyvalues where a keyvalue from target column
+ * with the same timestamp do not exist will be dropped.
+ *
* @param family name of target column family
* @param qualifier name of column qualifier
*/
public DependentColumnFilter(final byte [] family, final byte [] qualifier) {
this(family, qualifier, false);
}
-
+
/**
* Constructor for DependentColumn filter.
- * Keyvalues where a keyvalue from target column
- * with the same timestamp do not exist will be dropped.
- *
+ * Keyvalues where a keyvalue from target column
+ * with the same timestamp do not exist will be dropped.
+ *
* @param family name of dependent column family
* @param qualifier name of dependent qualifier
* @param dropDependentColumn whether the dependent columns keyvalues should be discarded
*/
public DependentColumnFilter(final byte [] family, final byte [] qualifier,
- final boolean dropDependentColumn) {
+ final boolean dropDependentColumn) {
this(family, qualifier, dropDependentColumn, CompareOp.NO_OP, null);
}
-
-
+
+
@Override
public boolean filterAllRemaining() {
return false;
@@ -108,17 +110,17 @@ public class DependentColumnFilter exten
@Override
public ReturnCode filterKeyValue(KeyValue v) {
// Check if the column and qualifier match
- if (!v.matchingColumn(this.columnFamily, this.columnQualifier)) {
- // include non-matches for the time being, they'll be discarded afterwards
- return ReturnCode.INCLUDE;
- }
- // If it doesn't pass the op, skip it
- if(comparator != null && doCompare(compareOp, comparator, v.getValue(), 0, v.getValueLength()))
- return ReturnCode.SKIP;
-
+ if (!v.matchingColumn(this.columnFamily, this.columnQualifier)) {
+ // include non-matches for the time being, they'll be discarded afterwards
+ return ReturnCode.INCLUDE;
+ }
+ // If it doesn't pass the op, skip it
+ if(comparator != null && doCompare(compareOp, comparator, v.getValue(), 0, v.getValueLength()))
+ return ReturnCode.SKIP;
+
stampSet.add(v.getTimestamp());
if(dropDependentColumn) {
- return ReturnCode.SKIP;
+ return ReturnCode.SKIP;
}
return ReturnCode.INCLUDE;
}
@@ -139,7 +141,7 @@ public class DependentColumnFilter exten
public boolean hasFilterRow() {
return true;
}
-
+
@Override
public boolean filterRow() {
return false;
@@ -152,22 +154,64 @@ public class DependentColumnFilter exten
@Override
public void reset() {
- stampSet.clear();
+ stampSet.clear();
+ }
+
+ public byte [] getFamily() {
+ return this.columnFamily;
+ }
+
+ public byte [] getQualifier() {
+ return this.columnQualifier;
+ }
+
+ public boolean getDropDependentColumn() {
+ return this.dropDependentColumn;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 2 ||
+ filterArguments.size() == 3 ||
+ filterArguments.size() == 5,
+ "Expected 2, 3 or 5 but got: %s", filterArguments.size());
+ if (filterArguments.size() == 2) {
+ byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
+ return new DependentColumnFilter(family, qualifier);
+
+ } else if (filterArguments.size() == 3) {
+ byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
+ boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
+ return new DependentColumnFilter(family, qualifier, dropDependentColumn);
+
+ } else if (filterArguments.size() == 5) {
+ byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
+ boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
+ CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(3));
+ WritableByteArrayComparable comparator = ParseFilter.createComparator(
+ ParseFilter.removeQuotesFromByteArray(filterArguments.get(4)));
+ return new DependentColumnFilter(family, qualifier, dropDependentColumn,
+ compareOp, comparator);
+ } else {
+ throw new IllegalArgumentException("Expected 2, 3 or 5 but got: " + filterArguments.size());
+ }
}
@Override
public void readFields(DataInput in) throws IOException {
- super.readFields(in);
+ super.readFields(in);
this.columnFamily = Bytes.readByteArray(in);
- if(this.columnFamily.length == 0) {
- this.columnFamily = null;
- }
-
+ if(this.columnFamily.length == 0) {
+ this.columnFamily = null;
+ }
+
this.columnQualifier = Bytes.readByteArray(in);
if(this.columnQualifier.length == 0) {
this.columnQualifier = null;
- }
-
+ }
+
this.dropDependentColumn = in.readBoolean();
}
@@ -176,7 +220,7 @@ public class DependentColumnFilter exten
super.write(out);
Bytes.writeByteArray(out, this.columnFamily);
Bytes.writeByteArray(out, this.columnQualifier);
- out.writeBoolean(this.dropDependentColumn);
+ out.writeBoolean(this.dropDependentColumn);
}
}
Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java Tue Oct 11 17:43:28 2011
@@ -0,0 +1,76 @@
+/**
+ * Copyright 2010 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.hbase.KeyValue;
+
+import java.util.ArrayList;
+
+/**
+ * This filter is used to filter based on the column family. It takes an
+ * operator (equal, greater, not equal, etc) and a byte [] comparator for the
+ * column family portion of a key.
+ * <p/>
+ * This filter can be wrapped with {@link org.apache.hadoop.hbase.filter.WhileMatchFilter} and {@link org.apache.hadoop.hbase.filter.SkipFilter}
+ * to add more control.
+ * <p/>
+ * Multiple filters can be combined using {@link org.apache.hadoop.hbase.filter.FilterList}.
+ * <p/>
+ * If an already known column family is looked for, use {@link org.apache.hadoop.hbase.client.Get#addFamily(byte[])}
+ * directly rather than a filter.
+ */
+public class FamilyFilter extends CompareFilter {
+ /**
+ * Writable constructor, do not use.
+ */
+ public FamilyFilter() {
+ }
+
+ /**
+ * Constructor.
+ *
+ * @param familyCompareOp the compare op for column family matching
+ * @param familyComparator the comparator for column family matching
+ */
+ public FamilyFilter(final CompareOp familyCompareOp,
+ final WritableByteArrayComparable familyComparator) {
+ super(familyCompareOp, familyComparator);
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ int familyLength = v.getFamilyLength();
+ if (familyLength > 0) {
+ if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+ v.getFamilyOffset(), familyLength)) {
+ return ReturnCode.SKIP;
+ }
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
+ return new FamilyFilter(compareOp, comparator);
+ }
+}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Tue Oct 11 17:43:28 2011
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.io.Writable;
import java.util.List;
+import java.util.ArrayList;
/**
* Interface for row and column filters directly applied within the regionserver.
@@ -45,7 +46,7 @@ import java.util.List;
*
* When implementing your own filters, consider inheriting {@link FilterBase} to help
* you reduce boilerplate.
- *
+ *
* @see FilterBase
*/
public interface Filter extends Writable {
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Tue Oct 11 17:43:28 2011
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import java.util.List;
+import java.util.ArrayList;
/**
* Abstract base class to help you implement new Filters. Common "ignore" or NOOP type
@@ -121,4 +122,13 @@ public abstract class FilterBase impleme
return null;
}
+ /**
+ * Given the filter's arguments it constructs the filter
+ * <p>
+ * @param filterArguments the filter's arguments
+ * @return constructed filter object
+ */
+ public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
+ throw new IllegalArgumentException("Method not implemented");
+ }
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java Tue Oct 11 17:43:28 2011
@@ -25,6 +25,9 @@ import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
import java.util.List;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* A filter that will only return the first KV from each row.
@@ -37,6 +40,12 @@ public class FirstKeyOnlyFilter extends
public FirstKeyOnlyFilter() {
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 0,
+ "Expected 0 but got: %s", filterArguments.size());
+ return new FirstKeyOnlyFilter();
+ }
+
public void reset() {
foundKV = false;
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Tue Oct 11 17:43:28 2011
@@ -27,6 +27,9 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* A Filter that stops after the given row. There is no "RowStopFilter" because
@@ -72,6 +75,13 @@ public class InclusiveStopFilter extends
return done;
}
+ public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 1,
+ "Expected 1 but got: %s", filterArguments.size());
+ byte [] stopRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ return new InclusiveStopFilter(stopRowKey);
+ }
+
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.stopRowKey);
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Tue Oct 11 17:43:28 2011
@@ -25,6 +25,10 @@ import java.io.IOException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
+
/**
* A filter that will only return the key component of each KV (the value will
* be rewritten as empty).
@@ -38,6 +42,12 @@ public class KeyOnlyFilter extends Filte
public KeyOnlyFilter() { this(false); }
public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 0,
+ "Expected: 0 but got: %s", filterArguments.size());
+ return new KeyOnlyFilter();
+ }
+
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
kv.convertToKeyOnly(this.lenAsVal);
Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Tue Oct 11 17:43:28 2011
@@ -0,0 +1,142 @@
+/*
+ * Copyright 2010 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.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.DataInput;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.TreeSet;
+import java.util.ArrayList;
+
+/**
+ * This filter is used for selecting only those keys with columns that matches
+ * a particular prefix. For example, if prefix is 'an', it will pass keys will
+ * columns like 'and', 'anti' but not keys with columns like 'ball', 'act'.
+ */
+public class MultipleColumnPrefixFilter extends FilterBase {
+ protected byte [] hint = null;
+ protected TreeSet<byte []> sortedPrefixes = createTreeSet();
+
+ public MultipleColumnPrefixFilter() {
+ super();
+ }
+
+ public MultipleColumnPrefixFilter(final byte [][] prefixes) {
+ if (prefixes != null) {
+ for (int i = 0; i < prefixes.length; i++) {
+ if (!sortedPrefixes.add(prefixes[i]))
+ throw new IllegalArgumentException ("prefixes must be distinct");
+ }
+ }
+ }
+
+ public byte [][] getPrefix() {
+ int count = 0;
+ byte [][] temp = new byte [sortedPrefixes.size()][];
+ for (byte [] prefixes : sortedPrefixes) {
+ temp [count++] = prefixes;
+ }
+ return temp;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue kv) {
+ if (sortedPrefixes.size() == 0 || kv.getBuffer() == null) {
+ return ReturnCode.INCLUDE;
+ } else {
+ return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+ }
+ }
+
+ public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) {
+ byte [] qualifier = Arrays.copyOfRange(buffer, qualifierOffset,
+ qualifierLength + qualifierOffset);
+ TreeSet<byte []> lesserOrEqualPrefixes =
+ (TreeSet<byte []>) sortedPrefixes.headSet(qualifier, true);
+
+ if (lesserOrEqualPrefixes.size() != 0) {
+ byte [] largestPrefixSmallerThanQualifier = lesserOrEqualPrefixes.last();
+
+ if (Bytes.startsWith(qualifier, largestPrefixSmallerThanQualifier)) {
+ return ReturnCode.INCLUDE;
+ }
+
+ if (lesserOrEqualPrefixes.size() == sortedPrefixes.size()) {
+ return ReturnCode.NEXT_ROW;
+ } else {
+ hint = sortedPrefixes.higher(largestPrefixSmallerThanQualifier);
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+ } else {
+ hint = sortedPrefixes.first();
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ byte [][] prefixes = new byte [filterArguments.size()][];
+ for (int i = 0 ; i < filterArguments.size(); i++) {
+ byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(i));
+ prefixes[i] = columnPrefix;
+ }
+ return new MultipleColumnPrefixFilter(prefixes);
+ }
+
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(sortedPrefixes.size());
+ for (byte [] element : sortedPrefixes) {
+ Bytes.writeByteArray(out, element);
+ }
+ }
+
+ public void readFields(DataInput in) throws IOException {
+ int x = in.readInt();
+ this.sortedPrefixes = createTreeSet();
+ for (int j = 0; j < x; j++) {
+ sortedPrefixes.add(Bytes.readByteArray(in));
+ }
+ }
+
+ public KeyValue getNextKeyHint(KeyValue kv) {
+ return KeyValue.createFirstOnRow(
+ kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+ kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
+ }
+
+ public TreeSet<byte []> createTreeSet() {
+ return new TreeSet<byte []>(new Comparator<Object>() {
+ @Override
+ public int compare (Object o1, Object o2) {
+ if (o1 == null || o2 == null)
+ throw new IllegalArgumentException ("prefixes can't be null");
+
+ byte [] b1 = (byte []) o1;
+ byte [] b2 = (byte []) o2;
+ return Bytes.compareTo (b1, 0, b1.length, b2, 0, b2.length);
+ }
+ });
+ }
+}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java Tue Oct 11 17:43:28 2011
@@ -25,7 +25,9 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
+import java.util.ArrayList;
+import com.google.common.base.Preconditions;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed rows is >
@@ -55,6 +57,7 @@ public class PageFilter extends FilterBa
* @param pageSize Maximum result size.
*/
public PageFilter(final long pageSize) {
+ Preconditions.checkArgument(pageSize >= 0, "must be positive %s", pageSize);
this.pageSize = pageSize;
}
@@ -71,6 +74,13 @@ public class PageFilter extends FilterBa
return this.rowsAccepted > this.pageSize;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 1,
+ "Expected 1 but got: %s", filterArguments.size());
+ long pageSize = ParseFilter.convertByteArrayToLong(filterArguments.get(0));
+ return new PageFilter(pageSize);
+ }
+
public void readFields(final DataInput in) throws IOException {
this.pageSize = in.readLong();
}
Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java Tue Oct 11 17:43:28 2011
@@ -0,0 +1,263 @@
+/**
+ * Copyright 2011 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.hbase.ipc.HRegionInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import org.apache.hadoop.hbase.filter.*;
+
+/**
+ * ParseConstants holds a bunch of constants related to parsing Filter Strings
+ * Used by {@link ParseFilter}
+ */
+public final class ParseConstants {
+
+ /**
+ * ASCII code for LPAREN
+ */
+ public static final int LPAREN = '(';
+
+ /**
+ * ASCII code for RPAREN
+ */
+ public static final int RPAREN = ')';
+
+ /**
+ * ASCII code for whitespace
+ */
+ public static final int WHITESPACE = ' ';
+
+ /**
+ * ASCII code for tab
+ */
+ public static final int TAB = '\t';
+
+ /**
+ * ASCII code for 'A'
+ */
+ public static final int A = 'A';
+
+ /**
+ * ASCII code for 'N'
+ */
+ public static final int N = 'N';
+
+ /**
+ * ASCII code for 'D'
+ */
+ public static final int D = 'D';
+
+ /**
+ * ASCII code for 'O'
+ */
+ public static final int O = 'O';
+
+ /**
+ * ASCII code for 'R'
+ */
+ public static final int R = 'R';
+
+ /**
+ * ASCII code for 'S'
+ */
+ public static final int S = 'S';
+
+ /**
+ * ASCII code for 'K'
+ */
+ public static final int K = 'K';
+
+ /**
+ * ASCII code for 'I'
+ */
+ public static final int I = 'I';
+
+ /**
+ * ASCII code for 'P'
+ */
+ public static final int P = 'P';
+
+ /**
+ * SKIP Array
+ */
+ public static final byte [] SKIP_ARRAY = new byte [ ] {'S', 'K', 'I', 'P'};
+ public static final ByteBuffer SKIP_BUFFER = ByteBuffer.wrap(SKIP_ARRAY);
+
+ /**
+ * ASCII code for 'W'
+ */
+ public static final int W = 'W';
+
+ /**
+ * ASCII code for 'H'
+ */
+ public static final int H = 'H';
+
+ /**
+ * ASCII code for 'L'
+ */
+ public static final int L = 'L';
+
+ /**
+ * ASCII code for 'E'
+ */
+ public static final int E = 'E';
+
+ /**
+ * WHILE Array
+ */
+ public static final byte [] WHILE_ARRAY = new byte [] {'W', 'H', 'I', 'L', 'E'};
+ public static final ByteBuffer WHILE_BUFFER = ByteBuffer.wrap(WHILE_ARRAY);
+
+ /**
+ * OR Array
+ */
+ public static final byte [] OR_ARRAY = new byte [] {'O','R'};
+ public static final ByteBuffer OR_BUFFER = ByteBuffer.wrap(OR_ARRAY);
+
+ /**
+ * AND Array
+ */
+ public static final byte [] AND_ARRAY = new byte [] {'A','N', 'D'};
+ public static final ByteBuffer AND_BUFFER = ByteBuffer.wrap(AND_ARRAY);
+
+ /**
+ * ASCII code for Backslash
+ */
+ public static final int BACKSLASH = '\\';
+
+ /**
+ * ASCII code for a single quote
+ */
+ public static final int SINGLE_QUOTE = '\'';
+
+ /**
+ * ASCII code for a comma
+ */
+ public static final int COMMA = ',';
+
+ /**
+ * LESS_THAN Array
+ */
+ public static final byte [] LESS_THAN_ARRAY = new byte [] {'<'};
+ public static final ByteBuffer LESS_THAN_BUFFER = ByteBuffer.wrap(LESS_THAN_ARRAY);
+
+ /**
+ * LESS_THAN_OR_EQUAL_TO Array
+ */
+ public static final byte [] LESS_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'<', '='};
+ public static final ByteBuffer LESS_THAN_OR_EQUAL_TO_BUFFER =
+ ByteBuffer.wrap(LESS_THAN_OR_EQUAL_TO_ARRAY);
+
+ /**
+ * GREATER_THAN Array
+ */
+ public static final byte [] GREATER_THAN_ARRAY = new byte [] {'>'};
+ public static final ByteBuffer GREATER_THAN_BUFFER = ByteBuffer.wrap(GREATER_THAN_ARRAY);
+
+ /**
+ * GREATER_THAN_OR_EQUAL_TO Array
+ */
+ public static final byte [] GREATER_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'>', '='};
+ public static final ByteBuffer GREATER_THAN_OR_EQUAL_TO_BUFFER =
+ ByteBuffer.wrap(GREATER_THAN_OR_EQUAL_TO_ARRAY);
+
+ /**
+ * EQUAL_TO Array
+ */
+ public static final byte [] EQUAL_TO_ARRAY = new byte [] {'='};
+ public static final ByteBuffer EQUAL_TO_BUFFER = ByteBuffer.wrap(EQUAL_TO_ARRAY);
+
+ /**
+ * NOT_EQUAL_TO Array
+ */
+ public static final byte [] NOT_EQUAL_TO_ARRAY = new byte [] {'!', '='};
+ public static final ByteBuffer NOT_EQUAL_TO_BUFFER = ByteBuffer.wrap(NOT_EQUAL_TO_ARRAY);
+
+ /**
+ * ASCII code for equal to (=)
+ */
+ public static final int EQUAL_TO = '=';
+
+ /**
+ * AND Byte Array
+ */
+ public static final byte [] AND = new byte [] {'A','N','D'};
+
+ /**
+ * OR Byte Array
+ */
+ public static final byte [] OR = new byte [] {'O', 'R'};
+
+ /**
+ * LPAREN Array
+ */
+ public static final byte [] LPAREN_ARRAY = new byte [] {'('};
+ public static final ByteBuffer LPAREN_BUFFER = ByteBuffer.wrap(LPAREN_ARRAY);
+
+ /**
+ * ASCII code for colon (:)
+ */
+ public static final int COLON = ':';
+
+ /**
+ * ASCII code for Zero
+ */
+ public static final int ZERO = '0';
+
+ /**
+ * ASCII code foe Nine
+ */
+ public static final int NINE = '9';
+
+ /**
+ * BinaryType byte array
+ */
+ public static final byte [] binaryType = new byte [] {'b','i','n','a','r','y'};
+
+ /**
+ * BinaryPrefixType byte array
+ */
+ public static final byte [] binaryPrefixType = new byte [] {'b','i','n','a','r','y',
+ 'p','r','e','f','i','x'};
+
+ /**
+ * RegexStringType byte array
+ */
+ public static final byte [] regexStringType = new byte [] {'r','e','g','e','x',
+ 's','t','r','i','n','g'};
+
+ /**
+ * SubstringType byte array
+ */
+ public static final byte [] substringType = new byte [] {'s','u','b','s','t','r','i','n','g'};
+
+ /**
+ * ASCII for Minus Sign
+ */
+ public static final int MINUS_SIGN = '-';
+
+ /**
+ * Package containing filters
+ */
+ public static final String FILTER_PACKAGE = "org.apache.hadoop.hbase.filter";
+}
Added: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (added)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java Tue Oct 11 17:43:28 2011
@@ -0,0 +1,844 @@
+/**
+ * Copyright 2011 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.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.TreeSet;
+import java.util.ArrayList;
+import java.util.Stack;
+import java.util.HashMap;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.ParseConstants;
+
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import java.lang.ArrayIndexOutOfBoundsException;
+import java.lang.ClassCastException;
+import java.lang.reflect.*;
+import java.util.EmptyStackException;
+
+/**
+ * This class allows a user to specify a filter via a string
+ * The string is parsed using the methods of this class and
+ * a filter object is constructed. This filter object is then wrapped
+ * in a scanner object which is then returned
+ * <p>
+ * More documentation on this Filter Language can be found here
+ * http://hbase.apache.org/book.html#thrift.filter-language
+ *
+ */
+public class ParseFilter {
+
+ private static HashMap<ByteBuffer, Integer> operatorPrecedenceHashMap;
+ private static HashMap<String, String> filterHashMap;
+
+ static {
+ // Registers all the filter supported by the Filter Language
+ filterHashMap = new HashMap<String, String>();
+ filterHashMap.put("KeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "KeyOnlyFilter");
+ filterHashMap.put("FirstKeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "FirstKeyOnlyFilter");
+ filterHashMap.put("PrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "PrefixFilter");
+ filterHashMap.put("ColumnPrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnPrefixFilter");
+ filterHashMap.put("MultipleColumnPrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "MultipleColumnPrefixFilter");
+ filterHashMap.put("ColumnCountGetFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnCountGetFilter");
+ filterHashMap.put("PageFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "PageFilter");
+ filterHashMap.put("ColumnPaginationFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnPaginationFilter");
+ filterHashMap.put("InclusiveStopFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "InclusiveStopFilter");
+ filterHashMap.put("TimestampsFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "TimestampsFilter");
+ filterHashMap.put("RowFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "RowFilter");
+ filterHashMap.put("FamilyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "FamilyFilter");
+ filterHashMap.put("QualifierFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "QualifierFilter");
+ filterHashMap.put("ValueFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ValueFilter");
+ filterHashMap.put("ColumnRangeFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnRangeFilter");
+ filterHashMap.put("SingleColumnValueFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "SingleColumnValueFilter");
+ filterHashMap.put("SingleColumnValueExcludeFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "SingleColumnValueExcludeFilter");
+ filterHashMap.put("DependentColumnFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "DependentColumnFilter");
+
+ // Creates the operatorPrecedenceHashMap
+ operatorPrecedenceHashMap = new HashMap<ByteBuffer, Integer>();
+ operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
+ operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
+ operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
+ operatorPrecedenceHashMap.put(ParseConstants.OR_BUFFER, 3);
+ }
+
+ /**
+ * Parses the filterString and constructs a filter using it
+ * <p>
+ * @param filterString filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseFilterString (String filterString)
+ throws CharacterCodingException {
+ return parseFilterString(Bytes.toBytes(filterString));
+ }
+
+ /**
+ * Parses the filterString and constructs a filter using it
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseFilterString (byte [] filterStringAsByteArray)
+ throws CharacterCodingException {
+ // stack for the operators and parenthesis
+ Stack <ByteBuffer> operatorStack = new Stack<ByteBuffer>();
+ // stack for the filter objects
+ Stack <Filter> filterStack = new Stack<Filter>();
+
+ Filter filter = null;
+ for (int i=0; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.LPAREN) {
+ // LPAREN found
+ operatorStack.push(ParseConstants.LPAREN_BUFFER);
+ } else if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[i] == ParseConstants.TAB) {
+ // WHITESPACE or TAB found
+ continue;
+ } else if (checkForOr(filterStringAsByteArray, i)) {
+ // OR found
+ i += ParseConstants.OR_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.OR_BUFFER);
+ operatorStack.push(ParseConstants.OR_BUFFER);
+ } else if (checkForAnd(filterStringAsByteArray, i)) {
+ // AND found
+ i += ParseConstants.AND_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.AND_BUFFER);
+ operatorStack.push(ParseConstants.AND_BUFFER);
+ } else if (checkForSkip(filterStringAsByteArray, i)) {
+ // SKIP found
+ i += ParseConstants.SKIP_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.SKIP_BUFFER);
+ operatorStack.push(ParseConstants.SKIP_BUFFER);
+ } else if (checkForWhile(filterStringAsByteArray, i)) {
+ // WHILE found
+ i += ParseConstants.WHILE_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.WHILE_BUFFER);
+ operatorStack.push(ParseConstants.WHILE_BUFFER);
+ } else if (filterStringAsByteArray[i] == ParseConstants.RPAREN) {
+ // RPAREN found
+ if (operatorStack.empty()) {
+ throw new IllegalArgumentException("Mismatched parenthesis");
+ }
+ ByteBuffer argumentOnTopOfStack = operatorStack.peek();
+ while (!(argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER))) {
+ filterStack.push(popArguments(operatorStack, filterStack));
+ if (operatorStack.empty()) {
+ throw new IllegalArgumentException("Mismatched parenthesis");
+ }
+ argumentOnTopOfStack = operatorStack.pop();
+ }
+ } else {
+ // SimpleFilterExpression found
+ byte [] filterSimpleExpression = extractFilterSimpleExpression(filterStringAsByteArray, i);
+ i+= (filterSimpleExpression.length - 1);
+ filter = parseSimpleFilterExpression(filterSimpleExpression);
+ filterStack.push(filter);
+ }
+ }
+
+ // Finished parsing filterString
+ while (!operatorStack.empty()) {
+ filterStack.push(popArguments(operatorStack, filterStack));
+ }
+ filter = filterStack.pop();
+ if (!filterStack.empty()) {
+ throw new IllegalArgumentException("Incorrect Filter String");
+ }
+ return filter;
+ }
+
+/**
+ * Extracts a simple filter expression from the filter string given by the user
+ * <p>
+ * A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg')
+ * The user given filter string can have many simpleFilterExpressions combined
+ * using operators.
+ * <p>
+ * This function extracts a simpleFilterExpression from the
+ * larger filterString given the start offset of the simpler expression
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param filterExpressionStartOffset start index of the simple filter expression
+ * @return byte array containing the simple filter expression
+ */
+ public byte [] extractFilterSimpleExpression (byte [] filterStringAsByteArray,
+ int filterExpressionStartOffset)
+ throws CharacterCodingException {
+ int quoteCount = 0;
+ for (int i=filterExpressionStartOffset; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
+ if (isQuoteUnescaped(filterStringAsByteArray, i)) {
+ quoteCount ++;
+ } else {
+ // To skip the next quote that has been escaped
+ i++;
+ }
+ }
+ if (filterStringAsByteArray[i] == ParseConstants.RPAREN && (quoteCount %2 ) == 0) {
+ byte [] filterSimpleExpression = new byte [i - filterExpressionStartOffset + 1];
+ Bytes.putBytes(filterSimpleExpression, 0, filterStringAsByteArray,
+ filterExpressionStartOffset, i-filterExpressionStartOffset + 1);
+ return filterSimpleExpression;
+ }
+ }
+ throw new IllegalArgumentException("Incorrect Filter String");
+ }
+
+/**
+ * Constructs a filter object given a simple filter expression
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseSimpleFilterExpression (byte [] filterStringAsByteArray)
+ throws CharacterCodingException {
+
+ String filterName = Bytes.toString(getFilterName(filterStringAsByteArray));
+ ArrayList<byte []> filterArguments = getFilterArguments(filterStringAsByteArray);
+ if (!filterHashMap.containsKey(filterName)) {
+ throw new IllegalArgumentException("Filter Name " + filterName + " not supported");
+ }
+ try {
+ filterName = filterHashMap.get(filterName);
+ Class c = Class.forName(filterName);
+ Class[] argTypes = new Class [] {ArrayList.class};
+ Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
+ return (Filter) m.invoke(null,filterArguments);
+ } catch (ClassNotFoundException e) {
+ e.printStackTrace();
+ } catch (NoSuchMethodException e) {
+ e.printStackTrace();
+ } catch (IllegalAccessException e) {
+ e.printStackTrace();
+ } catch (InvocationTargetException e) {
+ e.printStackTrace();
+ }
+ throw new IllegalArgumentException("Incorrect filter string " +
+ new String(filterStringAsByteArray));
+ }
+
+/**
+ * Returns the filter name given a simple filter expression
+ * <p>
+ * @param filterStringAsByteArray a simple filter expression
+ * @return name of filter in the simple filter expression
+ */
+ public static byte [] getFilterName (byte [] filterStringAsByteArray) {
+ int filterNameStartIndex = 0;
+ int filterNameEndIndex = 0;
+
+ for (int i=filterNameStartIndex; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.LPAREN ||
+ filterStringAsByteArray[i] == ParseConstants.WHITESPACE) {
+ filterNameEndIndex = i;
+ break;
+ }
+ }
+
+ if (filterNameEndIndex == 0) {
+ throw new IllegalArgumentException("Incorrect Filter Name");
+ }
+
+ byte [] filterName = new byte[filterNameEndIndex - filterNameStartIndex];
+ Bytes.putBytes(filterName, 0, filterStringAsByteArray, 0,
+ filterNameEndIndex - filterNameStartIndex);
+ return filterName;
+ }
+
+/**
+ * Returns the arguments of the filter from the filter string
+ * <p>
+ * @param filter_string filter string given by the user
+ * @return an ArrayList containing the arguments of the filter in the filter string
+ */
+ public static ArrayList<byte []> getFilterArguments (byte [] filterStringAsByteArray) {
+ int argumentListStartIndex = KeyValue.getDelimiter(filterStringAsByteArray, 0,
+ filterStringAsByteArray.length,
+ ParseConstants.LPAREN);
+ if (argumentListStartIndex == -1) {
+ throw new IllegalArgumentException("Incorrect argument list");
+ }
+
+ int argumentStartIndex = 0;
+ int argumentEndIndex = 0;
+ ArrayList<byte []> filterArguments = new ArrayList<byte []>();
+
+ for (int i = argumentListStartIndex + 1; i<filterStringAsByteArray.length; i++) {
+
+ if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[i] == ParseConstants.COMMA ||
+ filterStringAsByteArray[i] == ParseConstants.RPAREN) {
+ continue;
+ }
+
+ // The argument is in single quotes - for example 'prefix'
+ if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
+ argumentStartIndex = i;
+ for (int j = argumentStartIndex+1; j < filterStringAsByteArray.length; j++) {
+ if (filterStringAsByteArray[j] == ParseConstants.SINGLE_QUOTE) {
+ if (isQuoteUnescaped(filterStringAsByteArray,j)) {
+ argumentEndIndex = j;
+ i = j+1;
+ byte [] filterArgument = createUnescapdArgument(filterStringAsByteArray,
+ argumentStartIndex, argumentEndIndex);
+ filterArguments.add(filterArgument);
+ break;
+ } else {
+ // To jump over the second escaped quote
+ j++;
+ }
+ } else if (j == filterStringAsByteArray.length - 1) {
+ throw new IllegalArgumentException("Incorrect argument list");
+ }
+ }
+ } else {
+ // The argument is an integer, boolean, comparison operator like <, >, != etc
+ argumentStartIndex = i;
+ for (int j = argumentStartIndex; j < filterStringAsByteArray.length; j++) {
+ if (filterStringAsByteArray[j] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[j] == ParseConstants.COMMA ||
+ filterStringAsByteArray[j] == ParseConstants.RPAREN) {
+ argumentEndIndex = j - 1;
+ i = j;
+ byte [] filterArgument = new byte [argumentEndIndex - argumentStartIndex + 1];
+ Bytes.putBytes(filterArgument, 0, filterStringAsByteArray,
+ argumentStartIndex, argumentEndIndex - argumentStartIndex + 1);
+ filterArguments.add(filterArgument);
+ break;
+ } else if (j == filterStringAsByteArray.length - 1) {
+ throw new IllegalArgumentException("Incorrect argument list");
+ }
+ }
+ }
+ }
+ return filterArguments;
+ }
+
+/**
+ * This function is called while parsing the filterString and an operator is parsed
+ * <p>
+ * @param operatorStack the stack containing the operators and parenthesis
+ * @param filterStack the stack containing the filters
+ * @param operator the operator found while parsing the filterString
+ * @return returns the filterStack after evaluating the stack
+ */
+ public void reduce(Stack<ByteBuffer> operatorStack,
+ Stack<Filter> filterStack,
+ ByteBuffer operator) {
+ while (!operatorStack.empty() &&
+ !(ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek())) &&
+ hasHigherPriority(operatorStack.peek(), operator)) {
+ filterStack.push(popArguments(operatorStack, filterStack));
+ }
+ }
+
+ /**
+ * Pops an argument from the operator stack and the number of arguments required by the operator
+ * from the filterStack and evaluates them
+ * <p>
+ * @param operatorStack the stack containing the operators
+ * @param filterStack the stack containing the filters
+ * @return the evaluated filter
+ */
+ public static Filter popArguments (Stack<ByteBuffer> operatorStack, Stack <Filter> filterStack) {
+ ByteBuffer argumentOnTopOfStack = operatorStack.peek();
+
+ if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) {
+ // The top of the stack is an OR
+ try {
+ ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
+ while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) {
+ Filter filter = filterStack.pop();
+ listOfFilters.add(0, filter);
+ operatorStack.pop();
+ }
+ Filter filter = filterStack.pop();
+ listOfFilters.add(0, filter);
+ Filter orFilter = new FilterList(FilterList.Operator.MUST_PASS_ONE, listOfFilters);
+ return orFilter;
+ } catch (EmptyStackException e) {
+ throw new IllegalArgumentException("Incorrect input string - an OR needs two filters");
+ }
+
+ } else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) {
+ // The top of the stack is an AND
+ try {
+ ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
+ while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) {
+ Filter filter = filterStack.pop();
+ listOfFilters.add(0, filter);
+ operatorStack.pop();
+ }
+ Filter filter = filterStack.pop();
+ listOfFilters.add(0, filter);
+ Filter andFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL, listOfFilters);
+ return andFilter;
+ } catch (EmptyStackException e) {
+ throw new IllegalArgumentException("Incorrect input string - an AND needs two filters");
+ }
+
+ } else if (argumentOnTopOfStack.equals(ParseConstants.SKIP_BUFFER)) {
+ // The top of the stack is a SKIP
+ try {
+ Filter wrappedFilter = filterStack.pop();
+ Filter skipFilter = new SkipFilter(wrappedFilter);
+ operatorStack.pop();
+ return skipFilter;
+ } catch (EmptyStackException e) {
+ throw new IllegalArgumentException("Incorrect input string - a SKIP wraps a filter");
+ }
+
+ } else if (argumentOnTopOfStack.equals(ParseConstants.WHILE_BUFFER)) {
+ // The top of the stack is a WHILE
+ try {
+ Filter wrappedFilter = filterStack.pop();
+ Filter whileMatchFilter = new WhileMatchFilter(wrappedFilter);
+ operatorStack.pop();
+ return whileMatchFilter;
+ } catch (EmptyStackException e) {
+ throw new IllegalArgumentException("Incorrect input string - a WHILE wraps a filter");
+ }
+
+ } else if (argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
+ // The top of the stack is a LPAREN
+ try {
+ Filter filter = filterStack.pop();
+ operatorStack.pop();
+ return filter;
+ } catch (EmptyStackException e) {
+ throw new IllegalArgumentException("Incorrect Filter String");
+ }
+
+ } else {
+ throw new IllegalArgumentException("Incorrect arguments on operatorStack");
+ }
+ }
+
+/**
+ * Returns which operator has higher precedence
+ * <p>
+ * If a has higher precedence than b, it returns true
+ * If they have the same precedence, it returns false
+ */
+ public boolean hasHigherPriority(ByteBuffer a, ByteBuffer b) {
+ if ((operatorPrecedenceHashMap.get(a) - operatorPrecedenceHashMap.get(b)) < 0) {
+ return true;
+ }
+ return false;
+ }
+
+/**
+ * Removes the single quote escaping a single quote - thus it returns an unescaped argument
+ * <p>
+ * @param filterStringAsByteArray filter string given by user
+ * @param argumentStartIndex start index of the argument
+ * @param argumentEndIndex end index of the argument
+ * @return returns an unescaped argument
+ */
+ public static byte [] createUnescapdArgument (byte [] filterStringAsByteArray,
+ int argumentStartIndex, int argumentEndIndex) {
+ int unescapedArgumentLength = 2;
+ for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
+ unescapedArgumentLength ++;
+ if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE &&
+ i != (argumentEndIndex - 1) &&
+ filterStringAsByteArray[i+1] == ParseConstants.SINGLE_QUOTE) {
+ i++;
+ continue;
+ }
+ }
+
+ byte [] unescapedArgument = new byte [unescapedArgumentLength];
+ int count = 1;
+ unescapedArgument[0] = '\'';
+ for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
+ if (filterStringAsByteArray [i] == ParseConstants.SINGLE_QUOTE &&
+ i != (argumentEndIndex - 1) &&
+ filterStringAsByteArray [i+1] == ParseConstants.SINGLE_QUOTE) {
+ unescapedArgument[count++] = filterStringAsByteArray [i+1];
+ i++;
+ }
+ else {
+ unescapedArgument[count++] = filterStringAsByteArray [i];
+ }
+ }
+ unescapedArgument[unescapedArgumentLength - 1] = '\'';
+ return unescapedArgument;
+ }
+
+/**
+ * Checks if the current index of filter string we are on is the beginning of the keyword 'OR'
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param indexOfOr index at which an 'O' was read
+ * @return true if the keyword 'OR' is at the current index
+ */
+ public static boolean checkForOr (byte [] filterStringAsByteArray, int indexOfOr)
+ throws CharacterCodingException, ArrayIndexOutOfBoundsException {
+
+ try {
+ if (filterStringAsByteArray[indexOfOr] == ParseConstants.O &&
+ filterStringAsByteArray[indexOfOr+1] == ParseConstants.R &&
+ (filterStringAsByteArray[indexOfOr-1] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfOr-1] == ParseConstants.RPAREN) &&
+ (filterStringAsByteArray[indexOfOr+2] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfOr+2] == ParseConstants.LPAREN)) {
+ return true;
+ } else {
+ return false;
+ }
+ } catch (ArrayIndexOutOfBoundsException e) {
+ return false;
+ }
+ }
+
+/**
+ * Checks if the current index of filter string we are on is the beginning of the keyword 'AND'
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param indexOfAnd index at which an 'A' was read
+ * @return true if the keyword 'AND' is at the current index
+ */
+ public static boolean checkForAnd (byte [] filterStringAsByteArray, int indexOfAnd)
+ throws CharacterCodingException {
+
+ try {
+ if (filterStringAsByteArray[indexOfAnd] == ParseConstants.A &&
+ filterStringAsByteArray[indexOfAnd+1] == ParseConstants.N &&
+ filterStringAsByteArray[indexOfAnd+2] == ParseConstants.D &&
+ (filterStringAsByteArray[indexOfAnd-1] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfAnd-1] == ParseConstants.RPAREN) &&
+ (filterStringAsByteArray[indexOfAnd+3] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfAnd+3] == ParseConstants.LPAREN)) {
+ return true;
+ } else {
+ return false;
+ }
+ } catch (ArrayIndexOutOfBoundsException e) {
+ return false;
+ }
+ }
+
+/**
+ * Checks if the current index of filter string we are on is the beginning of the keyword 'SKIP'
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param indexOfSkip index at which an 'S' was read
+ * @return true if the keyword 'SKIP' is at the current index
+ */
+ public static boolean checkForSkip (byte [] filterStringAsByteArray, int indexOfSkip)
+ throws CharacterCodingException {
+
+ try {
+ if (filterStringAsByteArray[indexOfSkip] == ParseConstants.S &&
+ filterStringAsByteArray[indexOfSkip+1] == ParseConstants.K &&
+ filterStringAsByteArray[indexOfSkip+2] == ParseConstants.I &&
+ filterStringAsByteArray[indexOfSkip+3] == ParseConstants.P &&
+ (indexOfSkip == 0 ||
+ filterStringAsByteArray[indexOfSkip-1] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfSkip-1] == ParseConstants.RPAREN ||
+ filterStringAsByteArray[indexOfSkip-1] == ParseConstants.LPAREN) &&
+ (filterStringAsByteArray[indexOfSkip+4] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfSkip+4] == ParseConstants.LPAREN)) {
+ return true;
+ } else {
+ return false;
+ }
+ } catch (ArrayIndexOutOfBoundsException e) {
+ return false;
+ }
+ }
+
+/**
+ * Checks if the current index of filter string we are on is the beginning of the keyword 'WHILE'
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param indexOfWhile index at which an 'W' was read
+ * @return true if the keyword 'WHILE' is at the current index
+ */
+ public static boolean checkForWhile (byte [] filterStringAsByteArray, int indexOfWhile)
+ throws CharacterCodingException {
+
+ try {
+ if (filterStringAsByteArray[indexOfWhile] == ParseConstants.W &&
+ filterStringAsByteArray[indexOfWhile+1] == ParseConstants.H &&
+ filterStringAsByteArray[indexOfWhile+2] == ParseConstants.I &&
+ filterStringAsByteArray[indexOfWhile+3] == ParseConstants.L &&
+ filterStringAsByteArray[indexOfWhile+4] == ParseConstants.E &&
+ (indexOfWhile == 0 || filterStringAsByteArray[indexOfWhile-1] == ParseConstants.WHITESPACE
+ || filterStringAsByteArray[indexOfWhile-1] == ParseConstants.RPAREN ||
+ filterStringAsByteArray[indexOfWhile-1] == ParseConstants.LPAREN) &&
+ (filterStringAsByteArray[indexOfWhile+5] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[indexOfWhile+5] == ParseConstants.LPAREN)) {
+ return true;
+ } else {
+ return false;
+ }
+ } catch (ArrayIndexOutOfBoundsException e) {
+ return false;
+ }
+ }
+
+/**
+ * Returns a boolean indicating whether the quote was escaped or not
+ * <p>
+ * @param array byte array in which the quote was found
+ * @param quoteIndex index of the single quote
+ * @return returns true if the quote was unescaped
+ */
+ public static boolean isQuoteUnescaped (byte [] array, int quoteIndex) {
+ if (array == null) {
+ throw new IllegalArgumentException("isQuoteUnescaped called with a null array");
+ }
+
+ if (quoteIndex == array.length - 1 || array[quoteIndex+1] != ParseConstants.SINGLE_QUOTE) {
+ return true;
+ }
+ else {
+ return false;
+ }
+ }
+
+/**
+ * Takes a quoted byte array and converts it into an unquoted byte array
+ * For example: given a byte array representing 'abc', it returns a
+ * byte array representing abc
+ * <p>
+ * @param quotedByteArray the quoted byte array
+ * @return
+ */
+ public static byte [] removeQuotesFromByteArray (byte [] quotedByteArray) {
+ if (quotedByteArray == null ||
+ quotedByteArray.length < 2 ||
+ quotedByteArray[0] != ParseConstants.SINGLE_QUOTE ||
+ quotedByteArray[quotedByteArray.length - 1] != ParseConstants.SINGLE_QUOTE) {
+ throw new IllegalArgumentException("removeQuotesFromByteArray needs a quoted byte array");
+ } else {
+ byte [] targetString = new byte [quotedByteArray.length - 2];
+ Bytes.putBytes(targetString, 0, quotedByteArray, 1, quotedByteArray.length - 2);
+ return targetString;
+ }
+ }
+
+/**
+ * Converts an int expressed in a byte array to an actual int
+ * <p>
+ * This doesn't use Bytes.toInt because that assumes
+ * that there will be {@link #SIZEOF_INT} bytes available.
+ * <p>
+ * @param numberAsByteArray the int value expressed as a byte array
+ * @return the int value
+ */
+ public static int convertByteArrayToInt (byte [] numberAsByteArray) {
+
+ long tempResult = ParseFilter.convertByteArrayToLong(numberAsByteArray);
+
+ if (tempResult > Integer.MAX_VALUE) {
+ throw new IllegalArgumentException("Integer Argument too large");
+ } else if (tempResult < Integer.MIN_VALUE) {
+ throw new IllegalArgumentException("Integer Argument too small");
+ }
+
+ int result = (int) tempResult;
+ return result;
+ }
+
+/**
+ * Converts a long expressed in a byte array to an actual long
+ * <p>
+ * This doesn't use Bytes.toLong because that assumes
+ * that there will be {@link #SIZEOF_LONG} bytes available.
+ * <p>
+ * @param numberAsByteArray the long value expressed as a byte array
+ * @return the long value
+ */
+ public static long convertByteArrayToLong (byte [] numberAsByteArray) {
+ if (numberAsByteArray == null) {
+ throw new IllegalArgumentException("convertByteArrayToLong called with a null array");
+ }
+
+ int i = 0;
+ long result = 0;
+ boolean isNegative = false;
+
+ if (numberAsByteArray[i] == ParseConstants.MINUS_SIGN) {
+ i++;
+ isNegative = true;
+ }
+
+ while (i != numberAsByteArray.length) {
+ if (numberAsByteArray[i] < ParseConstants.ZERO ||
+ numberAsByteArray[i] > ParseConstants.NINE) {
+ throw new IllegalArgumentException("Byte Array should only contain digits");
+ }
+ result = result*10 + (numberAsByteArray[i] - ParseConstants.ZERO);
+ if (result < 0) {
+ throw new IllegalArgumentException("Long Argument too large");
+ }
+ i++;
+ }
+
+ if (isNegative) {
+ return -result;
+ } else {
+ return result;
+ }
+ }
+
+/**
+ * Converts a boolean expressed in a byte array to an actual boolean
+ *<p>
+ * This doesn't used Bytes.toBoolean because Bytes.toBoolean(byte [])
+ * assumes that 1 stands for true and 0 for false.
+ * Here, the byte array representing "true" and "false" is parsed
+ * <p>
+ * @param booleanAsByteArray the boolean value expressed as a byte array
+ * @return the boolean value
+ */
+ public static boolean convertByteArrayToBoolean (byte [] booleanAsByteArray) {
+ if (booleanAsByteArray == null) {
+ throw new IllegalArgumentException("convertByteArrayToBoolean called with a null array");
+ }
+
+ if (booleanAsByteArray.length == 4 &&
+ (booleanAsByteArray[0] == 't' || booleanAsByteArray[0] == 'T') &&
+ (booleanAsByteArray[1] == 'r' || booleanAsByteArray[1] == 'R') &&
+ (booleanAsByteArray[2] == 'u' || booleanAsByteArray[2] == 'U') &&
+ (booleanAsByteArray[3] == 'e' || booleanAsByteArray[3] == 'E')) {
+ return true;
+ }
+ else if (booleanAsByteArray.length == 5 &&
+ (booleanAsByteArray[0] == 'f' || booleanAsByteArray[0] == 'F') &&
+ (booleanAsByteArray[1] == 'a' || booleanAsByteArray[1] == 'A') &&
+ (booleanAsByteArray[2] == 'l' || booleanAsByteArray[2] == 'L') &&
+ (booleanAsByteArray[3] == 's' || booleanAsByteArray[3] == 'S') &&
+ (booleanAsByteArray[4] == 'e' || booleanAsByteArray[4] == 'E')) {
+ return false;
+ }
+ else {
+ throw new IllegalArgumentException("Incorrect Boolean Expression");
+ }
+ }
+
+/**
+ * Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
+ * <p>
+ * @param compareOpAsByteArray the comparatorOperator symbol as a byte array
+ * @return the Compare Operator
+ */
+ public static CompareFilter.CompareOp createCompareOp (byte [] compareOpAsByteArray) {
+ ByteBuffer compareOp = ByteBuffer.wrap(compareOpAsByteArray);
+ if (compareOp.equals(ParseConstants.LESS_THAN_BUFFER))
+ return CompareOp.LESS;
+ else if (compareOp.equals(ParseConstants.LESS_THAN_OR_EQUAL_TO_BUFFER))
+ return CompareOp.LESS_OR_EQUAL;
+ else if (compareOp.equals(ParseConstants.GREATER_THAN_BUFFER))
+ return CompareOp.GREATER;
+ else if (compareOp.equals(ParseConstants.GREATER_THAN_OR_EQUAL_TO_BUFFER))
+ return CompareOp.GREATER_OR_EQUAL;
+ else if (compareOp.equals(ParseConstants.NOT_EQUAL_TO_BUFFER))
+ return CompareOp.NOT_EQUAL;
+ else if (compareOp.equals(ParseConstants.EQUAL_TO_BUFFER))
+ return CompareOp.EQUAL;
+ else
+ throw new IllegalArgumentException("Invalid compare operator");
+ }
+
+/**
+ * Parses a comparator of the form comparatorType:comparatorValue form and returns a comparator
+ * <p>
+ * @param comparator the comparator in the form comparatorType:comparatorValue
+ * @return the parsed comparator
+ */
+ public static WritableByteArrayComparable createComparator (byte [] comparator) {
+ if (comparator == null)
+ throw new IllegalArgumentException("Incorrect Comparator");
+ byte [][] parsedComparator = ParseFilter.parseComparator(comparator);
+ byte [] comparatorType = parsedComparator[0];
+ byte [] comparatorValue = parsedComparator[1];
+
+
+ if (Bytes.equals(comparatorType, ParseConstants.binaryType))
+ return new BinaryComparator(comparatorValue);
+ else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
+ return new BinaryPrefixComparator(comparatorValue);
+ else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
+ return new RegexStringComparator(new String(comparatorValue));
+ else if (Bytes.equals(comparatorType, ParseConstants.substringType))
+ return new SubstringComparator(new String(comparatorValue));
+ else
+ throw new IllegalArgumentException("Incorrect comparatorType");
+ }
+
+/**
+ * Splits a column in comparatorType:comparatorValue form into separate byte arrays
+ * <p>
+ * @param comparator the comparator
+ * @return the parsed arguments of the comparator as a 2D byte array
+ */
+ public static byte [][] parseComparator (byte [] comparator) {
+ final int index = KeyValue.getDelimiter(comparator, 0, comparator.length, ParseConstants.COLON);
+ if (index == -1) {
+ throw new IllegalArgumentException("Incorrect comparator");
+ }
+
+ byte [][] result = new byte [2][0];
+ result[0] = new byte [index];
+ System.arraycopy(comparator, 0, result[0], 0, index);
+
+ final int len = comparator.length - (index + 1);
+ result[1] = new byte[len];
+ System.arraycopy(comparator, index + 1, result[1], 0, len);
+
+ return result;
+ }
+
+/**
+ * Return a Set of filters supported by the Filter Language
+ */
+ public Set<String> getSupportedFilters () {
+ return filterHashMap.keySet();
+ }
+}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java Tue Oct 11 17:43:28 2011
@@ -27,6 +27,9 @@ import java.io.DataOutput;
import java.io.IOException;
import java.io.DataInput;
import java.util.List;
+import java.util.ArrayList;
+
+import com.google.common.base.Preconditions;
/**
* Pass results that have same row prefix.
@@ -67,6 +70,13 @@ public class PrefixFilter extends Filter
return passedPrefix;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 1,
+ "Expected 1 but got: %s", filterArguments.size());
+ byte [] prefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ return new PrefixFilter(prefix);
+ }
+
public void write(DataOutput out) throws IOException {
Bytes.writeByteArray(out, this.prefix);
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Tue Oct 11 17:43:28 2011
@@ -23,6 +23,8 @@ package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Get;
+import java.util.ArrayList;
+
/**
* This filter is used to filter based on the column qualifier. It takes an
* operator (equal, greater, not equal, etc) and a byte [] comparator for the
@@ -65,4 +67,11 @@ public class QualifierFilter extends Com
}
return ReturnCode.INCLUDE;
}
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
+ return new QualifierFilter(compareOp, comparator);
+ }
}
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Tue Oct 11 17:43:28 2011
@@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import java.util.List;
+import java.util.ArrayList;
/**
* This filter is used to filter based on the key. It takes an operator
@@ -83,4 +84,11 @@ public class RowFilter extends CompareFi
public boolean filterRow() {
return this.filterOutRow;
}
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
+ return new RowFilter(compareOp, comparator);
+ }
}
\ No newline at end of file
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java Tue Oct 11 17:43:28 2011
@@ -23,6 +23,8 @@ package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import java.util.ArrayList;
+
/**
* A {@link Filter} that checks a single column value, but does not emit the
* tested column. This will enable a performance boost over
@@ -85,4 +87,18 @@ public class SingleColumnValueExcludeFil
}
return superRetCode;
}
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ SingleColumnValueFilter tempFilter = (SingleColumnValueFilter)
+ SingleColumnValueFilter.createFilterFromArguments(filterArguments);
+ SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter (
+ tempFilter.getFamily(), tempFilter.getQualifier(),
+ tempFilter.getOperator(), tempFilter.getComparator());
+
+ if (filterArguments.size() == 6) {
+ filter.setFilterIfMissing(tempFilter.getFilterIfMissing());
+ filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly());
+ }
+ return filter;
+ }
}