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 [2/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/
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java Tue Oct 11 17:43:28 2011
@@ -33,7 +33,9 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
+import java.util.ArrayList;
+import com.google.common.base.Preconditions;
/**
* This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp}
* operator (equal, greater, not equal, etc), and either a byte [] value or
@@ -244,6 +246,36 @@ public class SingleColumnValueFilter ext
this.latestVersionOnly = latestVersionOnly;
}
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6,
+ "Expected 4 or 6 but got: %s", filterArguments.size());
+ byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
+ byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
+ CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(2));
+ WritableByteArrayComparable comparator = ParseFilter.createComparator(
+ ParseFilter.removeQuotesFromByteArray(filterArguments.get(3)));
+
+ 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");
+ }
+ }
+
+ SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier,
+ compareOp, comparator);
+
+ if (filterArguments.size() == 6) {
+ boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4));
+ boolean latestVersionOnly = ParseFilter.convertByteArrayToBoolean(filterArguments.get(5));
+ filter.setFilterIfMissing(filterIfMissing);
+ filter.setLatestVersionOnly(latestVersionOnly);
+ }
+ return filter;
+ }
+
public void readFields(final DataInput in) throws IOException {
this.columnFamily = Bytes.readByteArray(in);
if(this.columnFamily.length == 0) {
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java Tue Oct 11 17:43:28 2011
@@ -4,9 +4,11 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
+import java.util.ArrayList;
import java.util.TreeSet;
import org.apache.hadoop.hbase.KeyValue;
+import com.google.common.base.Preconditions;
/**
* Filter that returns only cells whose timestamp (version) is
@@ -40,6 +42,9 @@ public class TimestampsFilter extends Fi
* @param timestamps
*/
public TimestampsFilter(List<Long> timestamps) {
+ for (Long timestamp : timestamps) {
+ Preconditions.checkArgument(timestamp >= 0, "must be positive %s", timestamp);
+ }
this.timestamps = new TreeSet<Long>(timestamps);
init();
}
@@ -70,6 +75,19 @@ public class TimestampsFilter extends Fi
return ReturnCode.SEEK_NEXT_USING_HINT;
}
+ public TreeSet<Long> getTimestamps() {
+ return this.timestamps;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList<Long> timestamps = new ArrayList<Long>();
+ for (int i = 0; i<filterArguments.size(); i++) {
+ long timestamp = ParseFilter.convertByteArrayToLong(filterArguments.get(i));
+ timestamps.add(timestamp);
+ }
+ return new TimestampsFilter(timestamps);
+ }
+
@Override
public void readFields(DataInput in) throws IOException {
int numTimestamps = in.readInt();
Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java Tue Oct 11 17:43:28 2011
@@ -22,6 +22,8 @@ package org.apache.hadoop.hbase.filter;
import org.apache.hadoop.hbase.KeyValue;
+import java.util.ArrayList;
+
/**
* This filter is used to filter based on column value. It takes an
* operator (equal, greater, not equal, etc) and a byte [] comparator for the
@@ -61,4 +63,11 @@ public class ValueFilter extends Compare
}
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 ValueFilter(compareOp, comparator);
+ }
}
Modified: hbase/branches/0.89/src/main/ruby/hbase.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/hbase.rb?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/hbase.rb (original)
+++ hbase/branches/0.89/src/main/ruby/hbase.rb Tue Oct 11 17:43:28 2011
@@ -54,6 +54,7 @@ module HBaseConstants
REPLICATION_SCOPE = "REPLICATION_SCOPE"
INTERVAL = 'INTERVAL'
CACHE = 'CACHE'
+ FILTER = 'FILTER'
# Load constants from hbase java API
def self.promote_constants(constants)
Modified: hbase/branches/0.89/src/main/ruby/hbase/table.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/hbase/table.rb?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/hbase/table.rb (original)
+++ hbase/branches/0.89/src/main/ruby/hbase/table.rb Tue Oct 11 17:43:28 2011
@@ -32,6 +32,7 @@ java_import org.apache.hadoop.hbase.clie
java_import org.apache.hadoop.hbase.client.Scan
java_import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
+java_import org.apache.hadoop.hbase.filter.ParseFilter
# Wrapper for org.apache.hadoop.hbase.client.HTable
@@ -240,7 +241,13 @@ module Hbase
end
columns.each { |c| scan.addColumns(c) }
- scan.setFilter(filter) if filter
+
+ unless filter.class == String
+ scan.setFilter(filter)
+ else
+ scan.setFilter(ParseFilter.new.parseFilterString(filter))
+ end
+
scan.setTimeStamp(timestamp) if timestamp
scan.setCacheBlocks(cache)
scan.setMaxVersions(versions) if versions > 1
Modified: hbase/branches/0.89/src/main/ruby/shell.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/shell.rb?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/shell.rb (original)
+++ hbase/branches/0.89/src/main/ruby/shell.rb Tue Oct 11 17:43:28 2011
@@ -217,6 +217,8 @@ Shell.load_command_group(
enable
exists
list
+ locate_regionservers
+ show_filters
]
)
Added: hbase/branches/0.89/src/main/ruby/shell/commands/locate_regionservers.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/shell/commands/locate_regionservers.rb?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/ruby/shell/commands/locate_regionservers.rb (added)
+++ hbase/branches/0.89/src/main/ruby/shell/commands/locate_regionservers.rb Tue Oct 11 17:43:28 2011
@@ -0,0 +1,69 @@
+#
+# 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.
+#
+
+java_import org.apache.hadoop.hbase.client.HTable
+java_import org.apache.hadoop.hbase.util.Bytes
+
+require 'set'
+
+module Shell
+ module Commands
+ class LocateRegionservers < Command
+ def help
+ return <<-EOF
+Lists the regionservers hosting the table along with their start and end keys.
+For example:
+ hbase> locate_regionservers 't1'
+If provided with a row key, it lists the regionserver hosting that key.
+For example:
+ hbase> locate_regionservers 't1', 'row_key'
+EOF
+ end
+
+ def command(table, key=nil)
+ now = Time.now
+
+ htable = HTable.new(table)
+ if key != nil then
+ regionLocation = htable.getRegionLocation(key).serverAddress.toString
+ formatter.row([regionLocation])
+ else
+ regionsInfo = htable.getRegionsInfo
+ hosting_regionservers = Array.new
+
+ regionsInfo.each do |hregioninfo, hserveraddress|
+ array = Array.new
+ address_name = hserveraddress.toString
+ start_key = Bytes::toStringBinary(hregioninfo.getStartKey)
+ end_key = Bytes::toStringBinary(hregioninfo.getEndKey)
+ array = "#{address_name}\t#{start_key}\t#{end_key}"
+ hosting_regionservers << array
+ end
+
+ hosting_regionservers.each do |regionserver|
+ formatter.row([regionserver])
+ end
+ end
+
+ formatter.footer(now)
+ end
+ end
+ end
+end
Modified: hbase/branches/0.89/src/main/ruby/shell/commands/scan.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/shell/commands/scan.rb?rev=1181945&r1=1181944&r2=1181945&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/ruby/shell/commands/scan.rb (original)
+++ hbase/branches/0.89/src/main/ruby/shell/commands/scan.rb Tue Oct 11 17:43:28 2011
@@ -23,23 +23,35 @@ module Shell
class Scan < Command
def help
return <<-EOF
- Scan a table; pass table name and optionally a dictionary of scanner
- specifications. Scanner specifications may include one or more of
- the following: LIMIT, STARTROW, STOPROW, TIMESTAMP, or COLUMNS. If
- no columns are specified, all columns will be scanned. To scan all
- members of a column family, leave the qualifier empty as in
- 'col_family:'. Examples:
-
- hbase> scan '.META.'
- hbase> scan '.META.', {COLUMNS => 'info:regioninfo'}
- hbase> scan 't1', {COLUMNS => ['c1', 'c2'], LIMIT => 10, STARTROW => 'xyz'}
-
- For experts, there is an additional option -- CACHE_BLOCKS -- which
- switches block caching for the scanner on (true) or off (false). By
- default it is enabled. Examples:
+Scan a table; pass table name and optionally a dictionary of scanner
+specifications. Scanner specifications may include one or more of
+the following: LIMIT, STARTROW, STOPROW, TIMESTAMP, or COLUMNS.
+
+If no columns are specified, all columns will be scanned. To scan all
+members of a column family, leave the qualifier empty as in
+'col_family:'.
+
+The filter can be specified in two ways:
+1. Using a filterString - more information on this is available at:
+http://hbase.apache.org/book.html#thrift.filter-language
+2. Using the entire package name of the filter.
+
+Examples:
+
+hbase> scan '.META.'
+hbase> scan '.META.', {COLUMNS => 'info:regioninfo'}
+hbase> scan 't1', {COLUMNS => ['c1', 'c2'], LIMIT => 10, STARTROW => 'xyz'}
+hbase> scan 't1', {FILTER => "(PrefixFilter ('row2') AND \
+ (QualifierFilter (>=, 'binary:xyz'))) OR \
+ (TimestampsFilter ( 123, 456))"}
+hbase> scan 't1', {FILTER => org.apache.hadoop.hbase.filter.KeyOnlyFilter.new()}
+
+For experts, there is an additional option -- CACHE_BLOCKS -- which
+switches block caching for the scanner on (true) or off (false). By
+default it is enabled. Examples:
- hbase> scan 't1', {COLUMNS => ['c1', 'c2'], CACHE_BLOCKS => false}
- EOF
+ hbase> scan 't1', {COLUMNS => ['c1', 'c2'], CACHE_BLOCKS => false}
+EOF
end
def command(table, args = {})
Added: hbase/branches/0.89/src/main/ruby/shell/commands/show_filters.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/ruby/shell/commands/show_filters.rb?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/main/ruby/shell/commands/show_filters.rb (added)
+++ hbase/branches/0.89/src/main/ruby/shell/commands/show_filters.rb Tue Oct 11 17:43:28 2011
@@ -0,0 +1,49 @@
+#
+# 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.
+#
+
+java_import org.apache.hadoop.hbase.filter.ParseFilter
+
+module Shell
+ module Commands
+ class ShowFilters < Command
+ def help
+ return <<-EOF
+Show all the filters in hbase. Example:
+
+ hbase> show_filters
+EOF
+ end
+
+ def command( )
+ now = Time.now
+ formatter.row(["Documentation on filters mentioned below can " +
+ "be found at: http://hbase.apache.org/" +
+ "book.html#thrift.filter-language"])
+
+ parseFilter = ParseFilter.new
+ supportedFilters = parseFilter.getSupportedFilters
+
+ supportedFilters.each do |filter|
+ formatter.row([filter])
+ end
+ end
+ end
+ end
+end
Added: hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java?rev=1181945&view=auto
==============================================================================
--- hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java (added)
+++ hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java Tue Oct 11 17:43:28 2011
@@ -0,0 +1,682 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueTestUtil;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * This class tests ParseFilter.java
+ * It tests the entire work flow from when a string is given by the user
+ * and how it is parsed to construct the corresponding Filter object
+ */
+public class TestParseFilter {
+
+ ParseFilter f;
+ Filter filter;
+
+ @Before
+ public void setUp() throws Exception {
+ f = new ParseFilter();
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ // Nothing to do.
+ }
+
+ @Test
+ public void testKeyOnlyFilter() throws IOException {
+ String filterString = "KeyOnlyFilter()";
+ doTestFilter(filterString, KeyOnlyFilter.class);
+
+ String filterString2 = "KeyOnlyFilter ('') ";
+ byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
+ try {
+ filter = f.parseFilterString(filterStringAsByteArray2);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testFirstKeyOnlyFilter() throws IOException {
+ String filterString = " FirstKeyOnlyFilter( ) ";
+ doTestFilter(filterString, FirstKeyOnlyFilter.class);
+
+ String filterString2 = " FirstKeyOnlyFilter ('') ";
+ byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2);
+ try {
+ filter = f.parseFilterString(filterStringAsByteArray2);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testPrefixFilter() throws IOException {
+ String filterString = " PrefixFilter('row' ) ";
+ PrefixFilter prefixFilter = doTestFilter(filterString, PrefixFilter.class);
+ byte [] prefix = prefixFilter.getPrefix();
+ assertEquals(new String(prefix), "row");
+
+
+ filterString = " PrefixFilter(row)";
+ try {
+ doTestFilter(filterString, PrefixFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testColumnPrefixFilter() throws IOException {
+ String filterString = " ColumnPrefixFilter('qualifier' ) ";
+ ColumnPrefixFilter columnPrefixFilter =
+ doTestFilter(filterString, ColumnPrefixFilter.class);
+ byte [] columnPrefix = columnPrefixFilter.getPrefix();
+ assertEquals(new String(columnPrefix), "qualifier");
+ }
+
+ @Test
+ public void testMultipleColumnPrefixFilter() throws IOException {
+ String filterString = " MultipleColumnPrefixFilter('qualifier1', 'qualifier2' ) ";
+ MultipleColumnPrefixFilter multipleColumnPrefixFilter =
+ doTestFilter(filterString, MultipleColumnPrefixFilter.class);
+ byte [][] prefixes = multipleColumnPrefixFilter.getPrefix();
+ assertEquals(new String(prefixes[0]), "qualifier1");
+ assertEquals(new String(prefixes[1]), "qualifier2");
+ }
+
+ @Test
+ public void testColumnCountGetFilter() throws IOException {
+ String filterString = " ColumnCountGetFilter(4)";
+ ColumnCountGetFilter columnCountGetFilter =
+ doTestFilter(filterString, ColumnCountGetFilter.class);
+ int limit = columnCountGetFilter.getLimit();
+ assertEquals(limit, 4);
+
+ filterString = " ColumnCountGetFilter('abc')";
+ try {
+ doTestFilter(filterString, ColumnCountGetFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+
+ filterString = " ColumnCountGetFilter(2147483648)";
+ try {
+ doTestFilter(filterString, ColumnCountGetFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testPageFilter() throws IOException {
+ String filterString = " PageFilter(4)";
+ PageFilter pageFilter =
+ doTestFilter(filterString, PageFilter.class);
+ long pageSize = pageFilter.getPageSize();
+ assertEquals(pageSize, 4);
+
+ filterString = " PageFilter('123')";
+ try {
+ doTestFilter(filterString, PageFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("PageFilter needs an int as an argument");
+ }
+ }
+
+ @Test
+ public void testColumnPaginationFilter() throws IOException {
+ String filterString = "ColumnPaginationFilter(4, 6)";
+ ColumnPaginationFilter columnPaginationFilter =
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ int limit = columnPaginationFilter.getLimit();
+ assertEquals(limit, 4);
+ int offset = columnPaginationFilter.getOffset();
+ assertEquals(offset, 6);
+
+ filterString = " ColumnPaginationFilter('124')";
+ try {
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("ColumnPaginationFilter needs two arguments");
+ }
+
+ filterString = " ColumnPaginationFilter('4' , '123a')";
+ try {
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("ColumnPaginationFilter needs two ints as arguments");
+ }
+
+ filterString = " ColumnPaginationFilter('4' , '-123')";
+ try {
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("ColumnPaginationFilter arguments should not be negative");
+ }
+ }
+
+ @Test
+ public void testInclusiveStopFilter() throws IOException {
+ String filterString = "InclusiveStopFilter ('row 3')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "row 3");
+ }
+
+
+ @Test
+ public void testTimestampsFilter() throws IOException {
+ String filterString = "TimestampsFilter(9223372036854775806, 6)";
+ TimestampsFilter timestampsFilter =
+ doTestFilter(filterString, TimestampsFilter.class);
+ TreeSet<Long> timestamps = timestampsFilter.getTimestamps();
+ assertEquals(timestamps.size(), 2);
+ assertTrue(timestamps.contains(new Long(6)));
+
+ filterString = "TimestampsFilter()";
+ timestampsFilter = doTestFilter(filterString, TimestampsFilter.class);
+ timestamps = timestampsFilter.getTimestamps();
+ assertEquals(timestamps.size(), 0);
+
+ filterString = "TimestampsFilter(9223372036854775808, 6)";
+ try {
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("Long Argument was too large");
+ }
+
+ filterString = "TimestampsFilter(-45, 6)";
+ try {
+ doTestFilter(filterString, ColumnPaginationFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("Timestamp Arguments should not be negative");
+ }
+ }
+
+ @Test
+ public void testRowFilter() throws IOException {
+ String filterString = "RowFilter ( =, 'binary:regionse')";
+ RowFilter rowFilter =
+ doTestFilter(filterString, RowFilter.class);
+ assertEquals(CompareFilter.CompareOp.EQUAL, rowFilter.getOperator());
+ assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
+ assertEquals("regionse", new String(binaryComparator.getValue()));
+ }
+
+ @Test
+ public void testFamilyFilter() throws IOException {
+ String filterString = "FamilyFilter(>=, 'binaryprefix:pre')";
+ FamilyFilter familyFilter =
+ doTestFilter(filterString, FamilyFilter.class);
+ assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, familyFilter.getOperator());
+ assertTrue(familyFilter.getComparator() instanceof BinaryPrefixComparator);
+ BinaryPrefixComparator binaryPrefixComparator =
+ (BinaryPrefixComparator) familyFilter.getComparator();
+ assertEquals("pre", new String(binaryPrefixComparator.getValue()));
+ }
+
+ @Test
+ public void testQualifierFilter() throws IOException {
+ String filterString = "QualifierFilter(=, 'regexstring:pre*')";
+ QualifierFilter qualifierFilter =
+ doTestFilter(filterString, QualifierFilter.class);
+ assertEquals(CompareFilter.CompareOp.EQUAL, qualifierFilter.getOperator());
+ assertTrue(qualifierFilter.getComparator() instanceof RegexStringComparator);
+ RegexStringComparator regexStringComparator =
+ (RegexStringComparator) qualifierFilter.getComparator();
+ assertEquals("pre*", new String(regexStringComparator.getValue()));
+ }
+
+ @Test
+ public void testValueFilter() throws IOException {
+ String filterString = "ValueFilter(!=, 'substring:pre')";
+ ValueFilter valueFilter =
+ doTestFilter(filterString, ValueFilter.class);
+ assertEquals(CompareFilter.CompareOp.NOT_EQUAL, valueFilter.getOperator());
+ assertTrue(valueFilter.getComparator() instanceof SubstringComparator);
+ SubstringComparator substringComparator =
+ (SubstringComparator) valueFilter.getComparator();
+ assertEquals("pre", new String(substringComparator.getValue()));
+ }
+
+ @Test
+ public void testColumnRangeFilter() throws IOException {
+ String filterString = "ColumnRangeFilter('abc', true, 'xyz', false)";
+ ColumnRangeFilter columnRangeFilter =
+ doTestFilter(filterString, ColumnRangeFilter.class);
+ assertEquals("abc", new String(columnRangeFilter.getMinColumn()));
+ assertEquals("xyz", new String(columnRangeFilter.getMaxColumn()));
+ assertTrue(columnRangeFilter.isMinColumnInclusive());
+ assertFalse(columnRangeFilter.isMaxColumnInclusive());
+ }
+
+ @Test
+ public void testDependentColumnFilter() throws IOException {
+ String filterString = "DependentColumnFilter('family', 'qualifier', true, =, 'binary:abc')";
+ DependentColumnFilter dependentColumnFilter =
+ doTestFilter(filterString, DependentColumnFilter.class);
+ assertEquals("family", new String(dependentColumnFilter.getFamily()));
+ assertEquals("qualifier", new String(dependentColumnFilter.getQualifier()));
+ assertTrue(dependentColumnFilter.getDropDependentColumn());
+ assertEquals(CompareFilter.CompareOp.EQUAL, dependentColumnFilter.getOperator());
+ assertTrue(dependentColumnFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator)dependentColumnFilter.getComparator();
+ assertEquals("abc", new String(binaryComparator.getValue()));
+ }
+
+ @Test
+ public void testSingleColumnValueFilter() throws IOException {
+ String filterString = "SingleColumnValueFilter " +
+ "('family', 'qualifier', >=, 'binary:a', true, false)";
+ SingleColumnValueFilter singleColumnValueFilter =
+ doTestFilter(filterString, SingleColumnValueFilter.class);
+ assertEquals("family", new String(singleColumnValueFilter.getFamily()));
+ assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
+ assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER_OR_EQUAL);
+ assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator) singleColumnValueFilter.getComparator();
+ assertEquals(new String(binaryComparator.getValue()), "a");
+ assertTrue(singleColumnValueFilter.getFilterIfMissing());
+ assertFalse(singleColumnValueFilter.getLatestVersionOnly());
+
+
+ filterString = "SingleColumnValueFilter ('family', 'qualifier', >, 'binaryprefix:a')";
+ singleColumnValueFilter = doTestFilter(filterString, SingleColumnValueFilter.class);
+ assertEquals("family", new String(singleColumnValueFilter.getFamily()));
+ assertEquals("qualifier", new String(singleColumnValueFilter.getQualifier()));
+ assertEquals(singleColumnValueFilter.getOperator(), CompareFilter.CompareOp.GREATER);
+ assertTrue(singleColumnValueFilter.getComparator() instanceof BinaryPrefixComparator);
+ BinaryPrefixComparator binaryPrefixComparator =
+ (BinaryPrefixComparator) singleColumnValueFilter.getComparator();
+ assertEquals(new String(binaryPrefixComparator.getValue()), "a");
+ assertFalse(singleColumnValueFilter.getFilterIfMissing());
+ assertTrue(singleColumnValueFilter.getLatestVersionOnly());
+ }
+
+ @Test
+ public void testSingleColumnValueExcludeFilter() throws IOException {
+ String filterString =
+ "SingleColumnValueExcludeFilter ('family', 'qualifier', <, 'binaryprefix:a')";
+ SingleColumnValueExcludeFilter singleColumnValueExcludeFilter =
+ doTestFilter(filterString, SingleColumnValueExcludeFilter.class);
+ assertEquals(singleColumnValueExcludeFilter.getOperator(), CompareFilter.CompareOp.LESS);
+ assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
+ assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
+ assertEquals(new String(singleColumnValueExcludeFilter.getComparator().getValue()), "a");
+ assertFalse(singleColumnValueExcludeFilter.getFilterIfMissing());
+ assertTrue(singleColumnValueExcludeFilter.getLatestVersionOnly());
+
+ filterString = "SingleColumnValueExcludeFilter " +
+ "('family', 'qualifier', <=, 'binaryprefix:a', true, false)";
+ singleColumnValueExcludeFilter =
+ doTestFilter(filterString, SingleColumnValueExcludeFilter.class);
+ assertEquals("family", new String(singleColumnValueExcludeFilter.getFamily()));
+ assertEquals("qualifier", new String(singleColumnValueExcludeFilter.getQualifier()));
+ assertEquals(singleColumnValueExcludeFilter.getOperator(),
+ CompareFilter.CompareOp.LESS_OR_EQUAL);
+ assertTrue(singleColumnValueExcludeFilter.getComparator() instanceof BinaryPrefixComparator);
+ BinaryPrefixComparator binaryPrefixComparator =
+ (BinaryPrefixComparator) singleColumnValueExcludeFilter.getComparator();
+ assertEquals(new String(binaryPrefixComparator.getValue()), "a");
+ assertTrue(singleColumnValueExcludeFilter.getFilterIfMissing());
+ assertFalse(singleColumnValueExcludeFilter.getLatestVersionOnly());
+ }
+
+ @Test
+ public void testSkipFilter() throws IOException {
+ String filterString = "SKIP ValueFilter( =, 'binary:0')";
+ SkipFilter skipFilter =
+ doTestFilter(filterString, SkipFilter.class);
+ assertTrue(skipFilter.getFilter() instanceof ValueFilter);
+ ValueFilter valueFilter = (ValueFilter) skipFilter.getFilter();
+
+ assertEquals(CompareFilter.CompareOp.EQUAL, valueFilter.getOperator());
+ assertTrue(valueFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator) valueFilter.getComparator();
+ assertEquals("0", new String(binaryComparator.getValue()));
+ }
+
+ @Test
+ public void testWhileFilter() throws IOException {
+ String filterString = " WHILE RowFilter ( !=, 'binary:row1')";
+ WhileMatchFilter whileMatchFilter =
+ doTestFilter(filterString, WhileMatchFilter.class);
+ assertTrue(whileMatchFilter.getFilter() instanceof RowFilter);
+ RowFilter rowFilter = (RowFilter) whileMatchFilter.getFilter();
+
+ assertEquals(CompareFilter.CompareOp.NOT_EQUAL, rowFilter.getOperator());
+ assertTrue(rowFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator) rowFilter.getComparator();
+ assertEquals("row1", new String(binaryComparator.getValue()));
+ }
+
+ @Test
+ public void testCompoundFilter1() throws IOException {
+ String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter())";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+ ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof PrefixFilter);
+ assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
+ PrefixFilter PrefixFilter = (PrefixFilter) filters.get(0);
+ byte [] prefix = PrefixFilter.getPrefix();
+ assertEquals(new String(prefix), "realtime");
+ FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
+ }
+
+ @Test
+ public void testCompoundFilter2() throws IOException {
+ String filterString = "(PrefixFilter('realtime') AND QualifierFilter (>=, 'binary:e'))" +
+ "OR FamilyFilter (=, 'binary:qualifier') ";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+ ArrayList<Filter> filterListFilters = (ArrayList<Filter>) filterList.getFilters();
+ assertTrue(filterListFilters.get(0) instanceof FilterList);
+ assertTrue(filterListFilters.get(1) instanceof FamilyFilter);
+ assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ONE);
+
+ filterList = (FilterList) filterListFilters.get(0);
+ FamilyFilter familyFilter = (FamilyFilter) filterListFilters.get(1);
+
+ filterListFilters = (ArrayList<Filter>)filterList.getFilters();
+ assertTrue(filterListFilters.get(0) instanceof PrefixFilter);
+ assertTrue(filterListFilters.get(1) instanceof QualifierFilter);
+ assertEquals(filterList.getOperator(), FilterList.Operator.MUST_PASS_ALL);
+
+ assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
+ assertTrue(familyFilter.getComparator() instanceof BinaryComparator);
+ BinaryComparator binaryComparator = (BinaryComparator) familyFilter.getComparator();
+ assertEquals("qualifier", new String(binaryComparator.getValue()));
+
+ PrefixFilter prefixFilter = (PrefixFilter) filterListFilters.get(0);
+ byte [] prefix = prefixFilter.getPrefix();
+ assertEquals(new String(prefix), "realtime");
+
+ QualifierFilter qualifierFilter = (QualifierFilter) filterListFilters.get(1);
+ assertEquals(CompareFilter.CompareOp.GREATER_OR_EQUAL, qualifierFilter.getOperator());
+ assertTrue(qualifierFilter.getComparator() instanceof BinaryComparator);
+ binaryComparator = (BinaryComparator) qualifierFilter.getComparator();
+ assertEquals("e", new String(binaryComparator.getValue()));
+ }
+
+ @Test
+ public void testCompoundFilter3() throws IOException {
+ String filterString = " ColumnPrefixFilter ('realtime')AND " +
+ "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+ ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof FilterList);
+ assertTrue(filters.get(1) instanceof SkipFilter);
+
+ filterList = (FilterList) filters.get(0);
+ SkipFilter skipFilter = (SkipFilter) filters.get(1);
+
+ filters = (ArrayList<Filter>) filterList.getFilters();
+ assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
+ assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
+
+ ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
+ byte [] columnPrefix = columnPrefixFilter.getPrefix();
+ assertEquals(new String(columnPrefix), "realtime");
+
+ FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
+
+ assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
+ FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
+
+ assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
+ assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
+ SubstringComparator substringComparator =
+ (SubstringComparator) familyFilter.getComparator();
+ assertEquals("hihi", new String(substringComparator.getValue()));
+ }
+
+ @Test
+ public void testCompoundFilter4() throws IOException {
+ String filterString = " ColumnPrefixFilter ('realtime') OR " +
+ "FirstKeyOnlyFilter() OR SKIP FamilyFilter(=, 'substring:hihi')";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+ ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof ColumnPrefixFilter);
+ assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
+ assertTrue(filters.get(2) instanceof SkipFilter);
+
+ ColumnPrefixFilter columnPrefixFilter = (ColumnPrefixFilter) filters.get(0);
+ FirstKeyOnlyFilter firstKeyOnlyFilter = (FirstKeyOnlyFilter) filters.get(1);
+ SkipFilter skipFilter = (SkipFilter) filters.get(2);
+
+ byte [] columnPrefix = columnPrefixFilter.getPrefix();
+ assertEquals(new String(columnPrefix), "realtime");
+
+ assertTrue(skipFilter.getFilter() instanceof FamilyFilter);
+ FamilyFilter familyFilter = (FamilyFilter) skipFilter.getFilter();
+
+ assertEquals(CompareFilter.CompareOp.EQUAL, familyFilter.getOperator());
+ assertTrue(familyFilter.getComparator() instanceof SubstringComparator);
+ SubstringComparator substringComparator =
+ (SubstringComparator) familyFilter.getComparator();
+ assertEquals("hihi", new String(substringComparator.getValue()));
+ }
+
+ @Test
+ public void testIncorrectCompareOperator() throws IOException {
+ String filterString = "RowFilter ('>>' , 'binary:region')";
+ try {
+ doTestFilter(filterString, RowFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("Incorrect compare operator >>");
+ }
+ }
+
+ @Test
+ public void testIncorrectComparatorType () throws IOException {
+ String filterString = "RowFilter ('>=' , 'binaryoperator:region')";
+ try {
+ doTestFilter(filterString, RowFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("Incorrect comparator type: binaryoperator");
+ }
+
+ filterString = "RowFilter ('>=' 'regexstring:pre*')";
+ try {
+ doTestFilter(filterString, RowFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("RegexStringComparator can only be used with EQUAL or NOT_EQUAL");
+ }
+
+ filterString = "SingleColumnValueFilter" +
+ " ('family', 'qualifier', '>=', 'substring:a', 'true', 'false')')";
+ try {
+ doTestFilter(filterString, RowFilter.class);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println("SubtringComparator can only be used with EQUAL or NOT_EQUAL");
+ }
+ }
+
+ @Test
+ public void testPrecedence1() throws IOException {
+ String filterString = " (PrefixFilter ('realtime')AND FirstKeyOnlyFilter()" +
+ " OR KeyOnlyFilter())";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+
+ ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof FilterList);
+ assertTrue(filters.get(1) instanceof KeyOnlyFilter);
+
+ filterList = (FilterList) filters.get(0);
+ filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof PrefixFilter);
+ assertTrue(filters.get(1) instanceof FirstKeyOnlyFilter);
+
+ PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
+ byte [] prefix = prefixFilter.getPrefix();
+ assertEquals(new String(prefix), "realtime");
+ }
+
+ @Test
+ public void testPrecedence2() throws IOException {
+ String filterString = " PrefixFilter ('realtime')AND SKIP FirstKeyOnlyFilter()" +
+ "OR KeyOnlyFilter()";
+ FilterList filterList =
+ doTestFilter(filterString, FilterList.class);
+ ArrayList<Filter> filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof FilterList);
+ assertTrue(filters.get(1) instanceof KeyOnlyFilter);
+
+ filterList = (FilterList) filters.get(0);
+ filters = (ArrayList<Filter>) filterList.getFilters();
+
+ assertTrue(filters.get(0) instanceof PrefixFilter);
+ assertTrue(filters.get(1) instanceof SkipFilter);
+
+ PrefixFilter prefixFilter = (PrefixFilter)filters.get(0);
+ byte [] prefix = prefixFilter.getPrefix();
+ assertEquals(new String(prefix), "realtime");
+
+ SkipFilter skipFilter = (SkipFilter)filters.get(1);
+ assertTrue(skipFilter.getFilter() instanceof FirstKeyOnlyFilter);
+ }
+
+ @Test
+ public void testUnescapedQuote1 () throws IOException {
+ String filterString = "InclusiveStopFilter ('row''3')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "row'3");
+ }
+
+ @Test
+ public void testUnescapedQuote2 () throws IOException {
+ String filterString = "InclusiveStopFilter ('row''3''')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "row'3'");
+ }
+
+ @Test
+ public void testUnescapedQuote3 () throws IOException {
+ String filterString = " InclusiveStopFilter ('''')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "'");
+ }
+
+ @Test
+ public void testIncorrectFilterString () throws IOException {
+ String filterString = "()";
+ byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
+ try {
+ filter = f.parseFilterString(filterStringAsByteArray);
+ assertTrue(false);
+ } catch (IllegalArgumentException e) {
+ System.out.println(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testBackslash1 () throws IOException {
+ String filterString = "InclusiveStopFilter ('\\a')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "\\a");
+ }
+
+ @Test
+ public void testBackslash2 () throws IOException {
+ String filterString = "InclusiveStopFilter ('abc\\''\\''''\\\na')";
+ InclusiveStopFilter inclusiveStopFilter =
+ doTestFilter(filterString, InclusiveStopFilter.class);
+ byte [] stopRowKey = inclusiveStopFilter.getStopRowKey();
+ assertEquals(new String(stopRowKey), "abc\\'\\''\\\na");
+ }
+
+ @Test
+ public void testCorrectFilterString () throws IOException {
+ String filterString = "(FirstKeyOnlyFilter())";
+ FirstKeyOnlyFilter firstKeyOnlyFilter =
+ doTestFilter(filterString, FirstKeyOnlyFilter.class);
+ }
+
+ private <T extends Filter> T doTestFilter(String filterString, Class<T> clazz)
+ throws IOException {
+ byte [] filterStringAsByteArray = Bytes.toBytes(filterString);
+ filter = f.parseFilterString(filterStringAsByteArray);
+ assertEquals(clazz, filter.getClass());
+ return clazz.cast(filter);
+ }
+}