Index: src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java (revision 0) +++ src/test/java/org/apache/hadoop/hbase/filter/TestParseFilter.java (revision 0) @@ -0,0 +1,733 @@ +/** + * 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( ) "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof KeyOnlyFilter); + + String filterString2 = "KeyOnlyFilter ('') "; + byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2); + try { + filter = f.parseFilterString(filterStringAsByteArray2); + assertTrue(filter instanceof KeyOnlyFilter); + } catch (IllegalArgumentException e) { + System.out.println(e.getMessage()); + } + } + + @Test + public void testFirstKeyOnlyFilter() throws IOException { + String filterString = " FirstKeyOnlyFilter( ) "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof FirstKeyOnlyFilter); + + String filterString2 = " FirstKeyOnlyFilter ('') "; + byte [] filterStringAsByteArray2 = Bytes.toBytes(filterString2); + try { + filter = f.parseFilterString(filterStringAsByteArray2); + assertTrue(filter instanceof FirstKeyOnlyFilter); + } catch (IllegalArgumentException e) { + System.out.println(e.getMessage()); + } + } + + @Test + public void testPrefixFilter() throws IOException { + String filterString = " PrefixFilter('row' ) "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof PrefixFilter); + + PrefixFilter prefixFilter = (PrefixFilter) filter; + byte [] prefix = prefixFilter.getPrefix(); + assertEquals(new String(prefix), "row"); + + filterString = " PrefixFilter(row)"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof PrefixFilter); + } catch (IllegalArgumentException e) { + System.out.println("PrefixFilter needs a quoted string"); + } + + } + + @Test + public void testColumnPrefixFilter() throws IOException { + String filterString = " ColumnPrefixFilter('qualifier' ) "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnPrefixFilter); + byte [] columnPrefix = ((ColumnPrefixFilter)filter).getPrefix(); + assertEquals(new String(columnPrefix), "qualifier"); + } + + @Test + public void testMultipleColumnPrefixFilter() throws IOException { + String filterString = " MultipleColumnPrefixFilter('qualifier1', 'qualifier2' ) "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof MultipleColumnPrefixFilter); + byte [][] prefixes = ((MultipleColumnPrefixFilter)filter).getPrefix(); + assertEquals(new String(prefixes[0]), "qualifier1"); + assertEquals(new String(prefixes[1]), "qualifier2"); + } + + @Test + public void testColumnCountGetFilter() throws IOException { + String filterString = " ColumnCountGetFilter(4)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnCountGetFilter); + int limit = ((ColumnCountGetFilter)filter).getLimit(); + assertEquals(limit, 4); + + filterString = " ColumnCountGetFilter('abc')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnCountGetFilter); + } catch (IllegalArgumentException e) { + System.out.println("ColumnCountGetFilter needs an int as an argument"); + } + + filterString = " ColumnCountGetFilter(2147483648)"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnCountGetFilter); + } catch (IllegalArgumentException e) { + System.out.println("Integer argument too large"); + } + } + + @Test + public void testPageFilter() throws IOException { + String filterString = " PageFilter(4)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof PageFilter); + + long pageSize = ((PageFilter)filter).getPageSize(); + assertEquals(pageSize, 4); + + filterString = " PageFilter('123')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof PageFilter); + } catch (IllegalArgumentException e) { + System.out.println("PageFilter needs an int as an argument"); + } + } + + @Test + public void testColumnPaginationFilter() throws IOException { + String filterString = "ColumnPaginationFilter(4, 6)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnPaginationFilter); + + int limit = ((ColumnPaginationFilter)filter).getLimit(); + assertEquals(limit, 4); + int offset = ((ColumnPaginationFilter)filter).getOffset(); + assertEquals(offset, 6); + + filterString = " ColumnPaginationFilter('124')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnPaginationFilter); + } catch (IllegalArgumentException e) { + System.out.println("ColumnPaginationFilter needs two arguments"); + } + + filterString = " ColumnPaginationFilter('4' , '123a')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnPaginationFilter); + } catch (IllegalArgumentException e) { + System.out.println("ColumnPaginationFilter needs two ints as arguments"); + } + + filterString = " ColumnPaginationFilter('4' , '-123')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnPaginationFilter); + } catch (IllegalArgumentException e) { + System.out.println("ColumnPaginationFilter arguments should not be negative"); + } + } + + @Test + public void testInclusiveStopFilter() throws IOException { + String filterString = "InclusiveStopFilter ('row 3')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof InclusiveStopFilter); + + byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey(); + assertEquals(new String(stopRowKey), "row 3"); + } + + + @Test + public void testTimestampsFilter() throws IOException { + String filterString = "TimestampsFilter(9223372036854775806, 6)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof TimestampsFilter); + + TreeSet timestamps = (TreeSet)((TimestampsFilter)filter).getTimestamps(); + assertEquals(timestamps.size(), 2); + assertTrue(timestamps.contains(new Long(6))); + + filterString = "TimestampsFilter()"; + filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof TimestampsFilter); + } + + @Test + public void testRowFilter() throws IOException { + String filterString = "RowFilter ( =, 'binary:regionse')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof RowFilter); + + RowFilter rowFilter = (RowFilter)filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof FamilyFilter); + + FamilyFilter familyFilter = (FamilyFilter)filter; + 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*')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof QualifierFilter); + + QualifierFilter qualifierFilter = (QualifierFilter) filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ValueFilter); + + ValueFilter valueFilter = (ValueFilter) filter; + 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)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof ColumnRangeFilter); + + ColumnRangeFilter columnRangeFilter = (ColumnRangeFilter) filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof DependentColumnFilter); + + DependentColumnFilter dependentColumnFilter = (DependentColumnFilter) filter; + assertEquals("family", new String(dependentColumnFilter.getFamily())); + assertEquals("qualifier", new String(dependentColumnFilter.getQualifier())); + assertTrue(dependentColumnFilter.getDropDepenedentColumn()); + 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)"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + + assertTrue(filter instanceof SingleColumnValueFilter); + + SingleColumnValueFilter singleColumnValueFilter = (SingleColumnValueFilter)filter; + 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')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + + assertTrue(filter instanceof SingleColumnValueFilter); + + singleColumnValueFilter = (SingleColumnValueFilter)filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof SingleColumnValueExcludeFilter); + + SingleColumnValueExcludeFilter singleColumnValueExcludeFilter = + (SingleColumnValueExcludeFilter)filter; + 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)"; + filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof SingleColumnValueExcludeFilter); + + singleColumnValueExcludeFilter = (SingleColumnValueExcludeFilter)filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof SkipFilter); + SkipFilter skipFilter = (SkipFilter) filter; + 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof WhileMatchFilter); + WhileMatchFilter whileMatchFilter = (WhileMatchFilter) filter; + 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())"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + FilterList filterList = (FilterList)filter; + ArrayList filters = (ArrayList) 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') "; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + + FilterList filterList = (FilterList)filter; + ArrayList filterListFilters = (ArrayList) 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)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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + FilterList filterList = (FilterList)filter; + ArrayList filters = (ArrayList) 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) 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + FilterList filterList = (FilterList)filter; + ArrayList filters = (ArrayList) 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')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof RowFilter); + } catch (IllegalArgumentException e) { + System.out.println("Incorrect compare operator >>"); + } + } + + @Test + public void testIncorrectComparatorType () throws IOException { + String filterString = "RowFilter ('>=' , 'binaryoperator:region')"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof RowFilter); + } catch (IllegalArgumentException e) { + System.out.println("Incorrect comparator type: binaryoperator"); + } + + filterString = "RowFilter ('>=' 'regexstring:pre*')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof RowFilter); + } catch (IllegalArgumentException e) { + System.out.println("RegexStringComparator can only be used with EQUAL or NOT_EQUAL"); + } + + filterString = "SingleColumnValueFilter" + + " ('family', 'qualifier', '>=', 'substring:a', 'true', 'false')')"; + filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof RowFilter); + } 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())"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + FilterList filterList = (FilterList)filter; + ArrayList filters = (ArrayList) filterList.getFilters(); + + assertTrue(filters.get(0) instanceof FilterList); + assertTrue(filters.get(1) instanceof KeyOnlyFilter); + + filterList = (FilterList) filters.get(0); + filters = (ArrayList) 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()"; + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + FilterList filterList = (FilterList)filter; + ArrayList filters = (ArrayList) filterList.getFilters(); + + assertTrue(filters.get(0) instanceof FilterList); + assertTrue(filters.get(1) instanceof KeyOnlyFilter); + + filterList = (FilterList) filters.get(0); + filters = (ArrayList) 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')"; + System.out.println(filterString); + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof InclusiveStopFilter); + + byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey(); + assertEquals(new String(stopRowKey), "row'3"); + } + + @Test + public void testUnescapedQuote2 () throws IOException { + String filterString = "InclusiveStopFilter ('row''3''')"; + System.out.println(filterString); + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof InclusiveStopFilter); + + byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey(); + assertEquals(new String(stopRowKey), "row'3'"); + } + + @Test + public void testUnescapedQuote3 () throws IOException { + String filterString = " InclusiveStopFilter ('''')"; + System.out.println(filterString); + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof InclusiveStopFilter); + + byte [] stopRowKey = ((InclusiveStopFilter)filter).getStopRowKey(); + assertEquals(new String(stopRowKey), "'"); + } + + @Test + public void testIncorrectFilterString () throws IOException { + String filterString = "()"; + System.out.println(filterString); + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + try { + filter = f.parseFilterString(filterStringAsByteArray); + } catch (IllegalArgumentException e) { + System.out.println(e.getMessage()); + } + + filterString = " OR KeyOnlyFilter() FirstKeyOnlyFilter()"; + System.out.println(filterString); + filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + } + + @Test + public void testCorrectFilterString () throws IOException { + String filterString = "(FirstKeyOnlyFilter())"; + System.out.println(filterString); + byte [] filterStringAsByteArray = Bytes.toBytes(filterString); + filter = f.parseFilterString(filterStringAsByteArray); + assertTrue(filter instanceof FirstKeyOnlyFilter); + } +} Index: src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (working copy) @@ -25,6 +25,8 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; +import java.util.ArrayList; + /** * A filter that will only return the key component of each KV (the value will * be rewritten as empty). @@ -44,6 +46,15 @@ return ReturnCode.INCLUDE; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 0) { + throw new IllegalArgumentException("Incorrect Arguments passed to KeyOnlyFilter. " + + "Expected: 0 but got: " + filterArguments.size()); + } + return new KeyOnlyFilter(); + } + public void write(DataOutput out) throws IOException { out.writeBoolean(this.lenAsVal); } Index: src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (working copy) @@ -27,6 +27,7 @@ import java.io.IOException; import java.io.DataInput; import java.util.List; +import java.util.ArrayList; /** * Pass results that have same row prefix. @@ -67,6 +68,17 @@ return passedPrefix; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 1) { + throw new IllegalArgumentException("Incorrect arguments passed to PrefixFilter. " + + "Expected: 1 but got: " + filterArguments.size()); + } + + byte [] prefix = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + return new PrefixFilter(prefix); + } + public void write(DataOutput out) throws IOException { Bytes.writeByteArray(out, this.prefix); } Index: src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java (revision 0) @@ -0,0 +1,258 @@ +/** + * 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 = '-'; +} \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (working copy) @@ -26,6 +26,8 @@ import java.io.DataInput; import java.util.List; +import java.util.ArrayList; + /** * A filter that will only return the first KV from each row. *

@@ -47,6 +49,16 @@ return ReturnCode.INCLUDE; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 0) { + throw new IllegalArgumentException("Incorrect Arguments passed to FirstKeyOnlyFilter. " + + "Expected: 0 but got: " + filterArguments.size()); + } + + return new FirstKeyOnlyFilter(); + } + public void write(DataOutput out) throws IOException { } Index: src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java (working copy) @@ -23,6 +23,8 @@ 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,20 @@ } return superRetCode; } + + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + SingleColumnValueFilter singleColumnValueFilter = new SingleColumnValueFilter(); + 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; + } } Index: src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java (working copy) @@ -22,6 +22,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayList; import org.apache.hadoop.hbase.KeyValue; @@ -46,6 +47,12 @@ public ColumnPaginationFilter(final int limit, final int offset) { + if (limit < 0) { + throw new IllegalArgumentException("Limit must not be negative"); + } + if (offset < 0) { + throw new IllegalArgumentException("Offset must not be negative"); + } this.limit = limit; this.offset = offset; } @@ -83,6 +90,17 @@ this.count = 0; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 2) { + throw new IllegalArgumentException("Incorrect Arguments passed to ColumnPaginationFilter. " + + "Expected: 2 but got: " + 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(); Index: src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (working copy) @@ -27,6 +27,7 @@ import java.io.DataOutput; import java.io.IOException; import java.util.List; +import java.util.ArrayList; /** * A Filter that stops after the given row. There is no "RowStopFilter" because @@ -72,6 +73,16 @@ return done; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 1) { + throw new IllegalArgumentException("Incorrect arguments passed to InclusiveStopFilter. " + + "Expected: 1 but got: " + filterArguments.size()); + } + byte [] stopRowKey = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + return new InclusiveStopFilter(stopRowKey); + } + public void write(DataOutput out) throws IOException { Bytes.writeByteArray(out, this.stopRowKey); } Index: src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java (working copy) @@ -26,6 +26,7 @@ import java.io.DataOutput; import java.io.IOException; import java.io.DataInput; +import java.util.ArrayList; /** * This filter is used for selecting only those keys with columns that are @@ -144,6 +145,26 @@ } @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 4) { + throw new IllegalArgumentException("Incorrect arguments passed to ColumnRangeFilter. " + + "Expected: 4 but got: " + filterArguments.size()); + } + + byte [] minColumn = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + boolean minColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(1)); + byte [] maxColumn = ParseFilter.convertByteArrayToString(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, // we will not be able to differentiate empty string and null Index: src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (working copy) @@ -22,6 +22,7 @@ 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 @@ -119,4 +120,8 @@ return null; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + throw new IllegalArgumentException("This function has not been implemented"); + } } Index: src/main/java/org/apache/hadoop/hbase/filter/FilterList.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (working copy) @@ -276,4 +276,10 @@ public KeyValue getNextKeyHint(KeyValue currentKV) { return null; } + + @Override + public Filter createFilterFromArguments(ArrayList filterArguments) { + throw new IllegalArgumentException("This function has not been implemented" + + "A FilterList can be created using the AND/OR operators)"); + } } \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (working copy) @@ -25,6 +25,7 @@ import java.io.DataOutput; import java.io.IOException; import java.util.List; +import java.util.ArrayList; /** * Implementation of Filter interface that limits results to a specific page @@ -55,6 +56,9 @@ * @param pageSize Maximum result size. */ public PageFilter(final long pageSize) { + if (pageSize < 0) { + throw new IllegalArgumentException("Page Size must not be negative"); + } this.pageSize = pageSize; } @@ -71,6 +75,16 @@ return this.rowsAccepted > this.pageSize; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 1) { + throw new IllegalArgumentException("Incorrect Arguments passed to PageFilter. " + + "Expected: 1 but got: " + 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(); } Index: src/main/java/org/apache/hadoop/hbase/filter/Filter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/Filter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/Filter.java (working copy) @@ -24,6 +24,7 @@ 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 @@ * * When implementing your own filters, consider inheriting {@link FilterBase} to help * you reduce boilerplate. - * + * * @see FilterBase */ public interface Filter extends Writable { @@ -148,4 +149,12 @@ * not sure which key to seek to next. */ public KeyValue getNextKeyHint(KeyValue currentKV); + + /** + * Given the filter's arguments it constructs the filter + *

+ * @param filterArguments the filter's arguments + * @return constructed filter object + */ + public Filter createFilterFromArguments (ArrayList filterArguments); } Index: src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (working copy) @@ -22,6 +22,8 @@ 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 @@ -64,4 +66,9 @@ } return ReturnCode.INCLUDE; } + + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + return super.createFilterFromArguments(filterArguments); + } } Index: src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java (working copy) @@ -27,7 +27,7 @@ import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; - +import java.util.ArrayList; /** * This is a generic filter to be used to filter by comparison. It takes an * operator (equal, greater, not equal, etc) and a byte [] comparator. @@ -108,23 +108,45 @@ offset + length)); switch (compareOp) { case LESS: + return compareResult < 0; + case LESS_OR_EQUAL: return compareResult <= 0; - case LESS_OR_EQUAL: - return compareResult < 0; case EQUAL: + return compareResult == 0; + case NOT_EQUAL: return compareResult != 0; - case NOT_EQUAL: - return compareResult == 0; case GREATER_OR_EQUAL: + return compareResult >= 0; + case GREATER: return compareResult > 0; - case GREATER: - return compareResult >= 0; default: throw new RuntimeException("Unknown Compare op " + compareOp.name()); } } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 2) { + throw new IllegalArgumentException("Incorrect Arguments passed to Compare Filter. " + + "Expected: 2 but got: " + filterArguments.size()); + } + + this.compareOp = ParseFilter.createCompareOp(filterArguments.get(0)); + this.comparator = ParseFilter.createComparator( + ParseFilter.convertByteArrayToString(filterArguments.get(1))); + + if (this.comparator instanceof RegexStringComparator || + this.comparator instanceof SubstringComparator) { + if (this.compareOp != CompareOp.EQUAL && + this.compareOp != CompareOp.NOT_EQUAL) { + throw new IllegalArgumentException ("A regexstring comparator and substring comparator" + + " can only be used with EQUAL and NOT_EQUAL"); + } + } + return this; + } + public void readFields(DataInput in) throws IOException { compareOp = CompareOp.valueOf(in.readUTF()); comparator = (WritableByteArrayComparable) Index: src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java (working copy) @@ -33,6 +33,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.ArrayList; /** * This filter is used to filter cells based on value. It takes a {@link CompareFilter.CompareOp} @@ -247,6 +248,39 @@ this.latestVersionOnly = latestVersionOnly; } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 4 && filterArguments.size() != 6) + throw new IllegalArgumentException("Incorrect Arguments passed to SingleColumnValueFilter. " + + "Expected: 4 or 6 but got: " + filterArguments.size()); + + byte [] family = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + byte [] qualifier = ParseFilter.convertByteArrayToString(filterArguments.get(1)); + CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(2)); + WritableByteArrayComparable comparator = ParseFilter.createComparator( + ParseFilter.convertByteArrayToString(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) { Index: src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java (working copy) @@ -25,6 +25,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.util.ArrayList; /** * Simple filter that returns first N columns on row only. @@ -45,6 +46,9 @@ } public ColumnCountGetFilter(final int n) { + if (n < 0) { + throw new IllegalArgumentException("Limit must not be negative"); + } this.limit = n; } @@ -69,6 +73,16 @@ } @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 1) { + throw new IllegalArgumentException("Incorrect Arguments passed to ColumnCountGetFilter. " + + "Expected: 1 but got: " + 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(); } Index: src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (working copy) @@ -21,9 +21,9 @@ package org.apache.hadoop.hbase.filter; 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 +83,9 @@ public boolean filterRow() { return this.filterOutRow; } + + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + return super.createFilterFromArguments(filterArguments); + } } \ No newline at end of file Index: src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (revision 0) +++ src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (revision 0) @@ -0,0 +1,814 @@ +/** + * 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 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.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 + * + * More documentation on this Filter Language can be found here: + * https://our.intern.facebook.com/intern/wiki/index.php/HBase/Filter_Language + * + */ +public class ParseFilter { + + private HashMap filterHashMap; + private HashMap operatorPrecedenceHashMap; + + /** + * Constructor + *

+ * Sets up the filterHashMap by registering all the filters + * Creates the operatorPrecedenceHashMap + */ + public ParseFilter() { + + filterHashMap = new HashMap(); + registerFilter("KeyOnlyFilter", new KeyOnlyFilter()); + registerFilter("FirstKeyOnlyFilter", new FirstKeyOnlyFilter()); + registerFilter("PrefixFilter", new PrefixFilter()); + registerFilter("ColumnPrefixFilter", new ColumnPrefixFilter()); + registerFilter("MultipleColumnPrefixFilter", new MultipleColumnPrefixFilter()); + registerFilter("ColumnCountGetFilter", new ColumnCountGetFilter()); + registerFilter("PageFilter", new PageFilter()); + registerFilter("ColumnPaginationFilter", new ColumnPaginationFilter()); + registerFilter("InclusiveStopFilter", new InclusiveStopFilter()); + registerFilter("TimestampsFilter", new TimestampsFilter()); + registerFilter("RowFilter", new RowFilter()); + registerFilter("FamilyFilter", new FamilyFilter()); + registerFilter("QualifierFilter", new QualifierFilter()); + registerFilter("ValueFilter", new ValueFilter()); + registerFilter("ColumnRangeFilter", new ColumnRangeFilter()); + registerFilter("SingleColumnValueFilter", new SingleColumnValueFilter()); + registerFilter("SingleColumnValueExcludeFilter", new SingleColumnValueExcludeFilter()); + registerFilter("DependentColumnFilter", new DependentColumnFilter()); + + operatorPrecedenceHashMap = new HashMap(); + 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 + *

+ * @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 operatorStack = new Stack(); + // stack for the filter objects + Stack filterStack = new Stack(); + + Filter filter = null; + for (int i=0; i + * A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg') + * The user given filter string can have many simpleFilterExpressions combined + * using operators. + *

+ * This function extracts a simpleFilterExpression from the + * larger filterString given the start offset of the simpler expression + *

+ * @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 + * @param filterStringAsByteArray filter string given by the user + * @return filter object we constructed + */ + public Filter parseSimpleFilterExpression (byte [] filterStringAsByteArray) + throws CharacterCodingException { + + byte [] filterName = getFilterName(filterStringAsByteArray); + ArrayList filterArguments = getFilterArguments(filterStringAsByteArray); + + if (filterHashMap.containsKey(ByteBuffer.wrap(filterName))) { + Filter filterSimpleExpression = filterHashMap.get(ByteBuffer.wrap(filterName)); + return filterSimpleExpression.createFilterFromArguments(filterArguments); + } else { + throw new IllegalArgumentException("Incorrect filter string " + + new String(filterStringAsByteArray)); + } + } + + /** + * Registers the filter + *

+ * Inserts a key-value composed of a ByteBuffer containing the name of the filter + * and an object of the filter into the HashMap + *

+ * @param filterName the name of the filter we wish to register in the filterHashMap + * @param filter an object of that filter + */ + private void registerFilter(String filterName, FilterBase filter) { + byte [] filterNameAsByteArray = Bytes.toBytes(filterName); + ByteBuffer filterNameAsByteBuffer = ByteBuffer.wrap(filterNameAsByteArray); + filterHashMap.put(filterNameAsByteBuffer, filter); + } + +/** + * Returns the filter name given a simple filter expression + *

+ * @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 + * @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 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 filterArguments = new ArrayList(); + + for (int i = argumentListStartIndex + 1; i, != 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 + *

+ * @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 operatorStack, + Stack 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 + *

+ * @param operatorStack the stack containing the operators + * @param filterStack the stack containing the filters + * @return the evaluated filter + */ + public static Filter popArguments (Stack operatorStack, Stack filterStack) { + ByteBuffer argumentOnTopOfStack = operatorStack.peek(); + + if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) { + // The top of the stack is an OR + try { + ArrayList listOfFilters = new ArrayList(); + 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 listOfFilters = new ArrayList(); + 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 + *

+ * 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 + *

+ * @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' + *

+ * @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' + *

+ * @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' + *

+ * @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' + *

+ * @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 + *

+ * @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' + *

+ * @param stringAsByteArray the quoted byte array + * @return + */ + public static byte [] convertByteArrayToString (byte [] stringAsByteArray) { + if (stringAsByteArray == null || + stringAsByteArray.length < 2 || + stringAsByteArray[0] != ParseConstants.SINGLE_QUOTE || + stringAsByteArray[stringAsByteArray.length - 1] != ParseConstants.SINGLE_QUOTE) { + throw new IllegalArgumentException("convertByteArrayToString needs a quotes byte array"); + } else { + byte [] targetString = new byte [stringAsByteArray.length - 2]; + Bytes.putBytes(targetString, 0, stringAsByteArray, 1, stringAsByteArray.length - 2); + return targetString; + } + } + +/** + * Converts an int expressed in a byte array to an actual int + *

+ * This doesn't use Bytes.toInt because that assumes + * that there will be {@link #SIZEOF_INT} bytes available. + *

+ * @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 + *

+ * This doesn't use Bytes.toLong because that assumes + * that there will be {@link #SIZEOF_LONG} bytes available. + *

+ * @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 + *

+ * 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 + *

+ * @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 + *

+ * @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 + *

+ * @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 + *

+ * @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; + } +} Index: src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java (working copy) @@ -26,6 +26,7 @@ import java.io.DataOutput; import java.io.IOException; import java.io.DataInput; +import java.util.ArrayList; /** * This filter is used for selecting only those keys with columns that matches @@ -78,6 +79,17 @@ } } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() != 1) { + throw new IllegalArgumentException("Incorrect Arguments passed to ColumnPrefixFilter. " + + "Expected: 1 but got: " + filterArguments.size()); + } + + byte [] columnPrefix = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + return new ColumnPrefixFilter(columnPrefix); + } + public void write(DataOutput out) throws IOException { Bytes.writeByteArray(out, this.prefix); } Index: src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java (working copy) @@ -22,6 +22,8 @@ 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,9 @@ } return ReturnCode.INCLUDE; } + + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + return super.createFilterFromArguments(filterArguments); + } } Index: src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (working copy) @@ -23,6 +23,8 @@ 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,9 @@ } return ReturnCode.INCLUDE; } + + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + return super.createFilterFromArguments(filterArguments); + } } Index: src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (working copy) @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.ArrayList; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.util.Bytes; @@ -120,6 +121,10 @@ return this.dropDependentColumn; } + public boolean getDropDepenedentColumn() { + return this.dropDependentColumn; + } + @Override public boolean filterAllRemaining() { return false; @@ -169,25 +174,52 @@ public boolean filterRowKey(byte[] buffer, int offset, int length) { return false; } - @Override public void reset() { - stampSet.clear(); + stampSet.clear(); } @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + if (filterArguments.size() == 2) { + byte [] family = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + byte [] qualifier = ParseFilter.convertByteArrayToString(filterArguments.get(1)); + return new DependentColumnFilter(family, qualifier); + + } else if (filterArguments.size() == 3) { + byte [] family = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + byte [] qualifier = ParseFilter.convertByteArrayToString(filterArguments.get(1)); + boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2)); + return new DependentColumnFilter(family, qualifier, dropDependentColumn); + + } else if (filterArguments.size() == 5) { + byte [] family = ParseFilter.convertByteArrayToString(filterArguments.get(0)); + byte [] qualifier = ParseFilter.convertByteArrayToString(filterArguments.get(1)); + boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2)); + CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(3)); + WritableByteArrayComparable comparator = ParseFilter.createComparator( + ParseFilter.convertByteArrayToString(filterArguments.get(4))); + return new DependentColumnFilter(family, qualifier, dropDependentColumn, + compareOp, comparator); + } else { + throw new IllegalArgumentException("Incorrect Arguments passed to DependentColumnFilter. " + + "Expected: 2, 3 or 5 but got: " + filterArguments.size()); + } + } + + @Override public void readFields(DataInput in) throws IOException { super.readFields(in); this.columnFamily = Bytes.readByteArray(in); 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(); } Index: src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java (working copy) @@ -6,6 +6,7 @@ import java.util.ArrayList; import java.util.List; import java.util.TreeSet; +import java.util.ArrayList; import org.apache.hadoop.hbase.KeyValue; @@ -41,6 +42,11 @@ * @param timestamps */ public TimestampsFilter(List timestamps) { + for (Long timestamp : timestamps) { + if (timestamp < 0) { + throw new IllegalArgumentException("Timestamps must not be negative"); + } + } this.timestamps = new TreeSet(timestamps); init(); } @@ -81,6 +87,16 @@ } @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + ArrayList timestamps = new ArrayList(); + for (int i = 0; i(); Index: src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (revision 1154088) +++ src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (working copy) @@ -26,6 +26,7 @@ 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 @@ -92,6 +93,16 @@ } } + @Override + public Filter createFilterFromArguments (ArrayList filterArguments) { + byte [][] prefixes = new byte [filterArguments.size()][]; + for (int i = 0 ; i < filterArguments.size(); i++) { + byte [] columnPrefix = ParseFilter.convertByteArrayToString(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) {