HBASE-4176 Exposing HBase Filters to the Thrift API
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1159067 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4e403eb78b
commit
2bf98a61b4
|
@ -429,6 +429,7 @@ Release 0.91.0 - Unreleased
|
|||
HBASE-2233 Support both Hadoop 0.20 and 0.22
|
||||
HBASE-3857 Change the HFile Format (Mikhail & Liyin)
|
||||
HBASE-4114 Metrics for HFile HDFS block locality (Ming Ma)
|
||||
HBASE-4176 Exposing HBase Filters to the Thrift API (Anirudh Todi)
|
||||
|
||||
Release 0.90.5 - Unreleased
|
||||
|
||||
|
|
|
@ -25,6 +25,9 @@ import org.apache.hadoop.hbase.KeyValue;
|
|||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Simple filter that returns first N columns on row only.
|
||||
|
@ -45,6 +48,7 @@ public class ColumnCountGetFilter extends FilterBase {
|
|||
}
|
||||
|
||||
public ColumnCountGetFilter(final int n) {
|
||||
Preconditions.checkArgument(n >= 0, "limit be positive %s", n);
|
||||
this.limit = n;
|
||||
}
|
||||
|
||||
|
@ -68,6 +72,13 @@ public class ColumnCountGetFilter extends FilterBase {
|
|||
this.count = 0;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
|
||||
return new ColumnCountGetFilter(limit);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
this.limit = in.readInt();
|
||||
|
|
|
@ -22,8 +22,10 @@ package org.apache.hadoop.hbase.filter;
|
|||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A filter, based on the ColumnCountGetFilter, takes two arguments: limit and offset.
|
||||
|
@ -46,6 +48,8 @@ public class ColumnPaginationFilter extends FilterBase
|
|||
|
||||
public ColumnPaginationFilter(final int limit, final int offset)
|
||||
{
|
||||
Preconditions.checkArgument(limit >= 0, "limit must be positive %s", limit);
|
||||
Preconditions.checkArgument(offset >= 0, "offset must be positive %s", offset);
|
||||
this.limit = limit;
|
||||
this.offset = offset;
|
||||
}
|
||||
|
@ -83,6 +87,14 @@ public class ColumnPaginationFilter extends FilterBase
|
|||
this.count = 0;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 2,
|
||||
"Expected 2 but got: %s", filterArguments.size());
|
||||
int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0));
|
||||
int offset = ParseFilter.convertByteArrayToInt(filterArguments.get(1));
|
||||
return new ColumnPaginationFilter(limit, offset);
|
||||
}
|
||||
|
||||
public void readFields(DataInput in) throws IOException
|
||||
{
|
||||
this.limit = in.readInt();
|
||||
|
|
|
@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.DataInput;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* This filter is used for selecting only those keys with columns that matches
|
||||
|
@ -78,6 +81,13 @@ public class ColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
return new ColumnPrefixFilter(columnPrefix);
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
Bytes.writeByteArray(out, this.prefix);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,9 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.io.DataInput;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* This filter is used for selecting only those keys with columns that are
|
||||
|
@ -143,6 +146,22 @@ public class ColumnRangeFilter extends FilterBase {
|
|||
return ReturnCode.NEXT_ROW;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 4,
|
||||
"Expected 4 but got: %s", filterArguments.size());
|
||||
byte [] minColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
boolean minColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(1));
|
||||
byte [] maxColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(2));
|
||||
boolean maxColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(3));
|
||||
|
||||
if (minColumn.length == 0)
|
||||
minColumn = null;
|
||||
if (maxColumn.length == 0)
|
||||
maxColumn = null;
|
||||
return new ColumnRangeFilter(minColumn, minColumnInclusive,
|
||||
maxColumn, maxColumnInclusive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
// need to write out a flag for null value separately. Otherwise,
|
||||
|
|
|
@ -27,7 +27,9 @@ import java.io.DataInput;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
/**
|
||||
* This is a generic filter to be used to filter by comparison. It takes an
|
||||
* operator (equal, greater, not equal, etc) and a byte [] comparator.
|
||||
|
@ -125,6 +127,27 @@ public abstract class CompareFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static ArrayList extractArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 2,
|
||||
"Expected 2 but got: %s", filterArguments.size());
|
||||
CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(0));
|
||||
WritableByteArrayComparable comparator = ParseFilter.createComparator(
|
||||
ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)));
|
||||
|
||||
if (comparator instanceof RegexStringComparator ||
|
||||
comparator instanceof SubstringComparator) {
|
||||
if (compareOp != CompareOp.EQUAL &&
|
||||
compareOp != CompareOp.NOT_EQUAL) {
|
||||
throw new IllegalArgumentException ("A regexstring comparator and substring comparator" +
|
||||
" can only be used with EQUAL and NOT_EQUAL");
|
||||
}
|
||||
}
|
||||
ArrayList arguments = new ArrayList();
|
||||
arguments.add(compareOp);
|
||||
arguments.add(comparator);
|
||||
return arguments;
|
||||
}
|
||||
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
compareOp = CompareOp.valueOf(in.readUTF());
|
||||
comparator = (WritableByteArrayComparable)
|
||||
|
|
|
@ -26,10 +26,13 @@ import java.util.HashSet;
|
|||
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;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A filter for adding inter-column timestamp matching
|
||||
* Only cells with a correspondingly timestamped entry in
|
||||
|
@ -120,6 +123,10 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
return this.dropDependentColumn;
|
||||
}
|
||||
|
||||
public boolean getDropDependentColumn() {
|
||||
return this.dropDependentColumn;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean filterAllRemaining() {
|
||||
return false;
|
||||
|
@ -169,12 +176,41 @@ public class DependentColumnFilter extends CompareFilter {
|
|||
public boolean filterRowKey(byte[] buffer, int offset, int length) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
stampSet.clear();
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 2 ||
|
||||
filterArguments.size() == 3 ||
|
||||
filterArguments.size() == 5,
|
||||
"Expected 2, 3 or 5 but got: %s", filterArguments.size());
|
||||
if (filterArguments.size() == 2) {
|
||||
byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
|
||||
return new DependentColumnFilter(family, qualifier);
|
||||
|
||||
} else if (filterArguments.size() == 3) {
|
||||
byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
|
||||
boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
|
||||
return new DependentColumnFilter(family, qualifier, dropDependentColumn);
|
||||
|
||||
} else if (filterArguments.size() == 5) {
|
||||
byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1));
|
||||
boolean dropDependentColumn = ParseFilter.convertByteArrayToBoolean(filterArguments.get(2));
|
||||
CompareOp compareOp = ParseFilter.createCompareOp(filterArguments.get(3));
|
||||
WritableByteArrayComparable comparator = ParseFilter.createComparator(
|
||||
ParseFilter.removeQuotesFromByteArray(filterArguments.get(4)));
|
||||
return new DependentColumnFilter(family, qualifier, dropDependentColumn,
|
||||
compareOp, comparator);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Expected 2, 3 or 5 but got: " + filterArguments.size());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFields(DataInput in) throws IOException {
|
||||
super.readFields(in);
|
||||
|
|
|
@ -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 the column family. It takes an
|
||||
* operator (equal, greater, not equal, etc) and a byte [] comparator for the
|
||||
|
@ -64,4 +66,11 @@ public class FamilyFilter extends CompareFilter {
|
|||
}
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
|
||||
CompareOp compareOp = (CompareOp)arguments.get(0);
|
||||
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
|
||||
return new FamilyFilter(compareOp, comparator);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.filter;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* Abstract base class to help you implement new Filters. Common "ignore" or NOOP type
|
||||
|
@ -119,4 +120,13 @@ public abstract class FilterBase implements Filter {
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Given the filter's arguments it constructs the filter
|
||||
* <p>
|
||||
* @param filterArguments the filter's arguments
|
||||
* @return constructed filter object
|
||||
*/
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
throw new IllegalArgumentException("This method has not been implemented");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,9 @@ import java.io.DataOutput;
|
|||
import java.io.IOException;
|
||||
import java.io.DataInput;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A filter that will only return the first KV from each row.
|
||||
|
@ -47,6 +50,12 @@ public class FirstKeyOnlyFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 0,
|
||||
"Expected 0 but got: %s", filterArguments.size());
|
||||
return new FirstKeyOnlyFilter();
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
}
|
||||
|
||||
|
|
|
@ -27,6 +27,9 @@ import java.io.DataInput;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A Filter that stops after the given row. There is no "RowStopFilter" because
|
||||
|
@ -72,6 +75,13 @@ public class InclusiveStopFilter extends FilterBase {
|
|||
return done;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments (ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
byte [] stopRowKey = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
return new InclusiveStopFilter(stopRowKey);
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
Bytes.writeByteArray(out, this.stopRowKey);
|
||||
}
|
||||
|
|
|
@ -25,6 +25,10 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A filter that will only return the key component of each KV (the value will
|
||||
* be rewritten as empty).
|
||||
|
@ -44,6 +48,12 @@ public class KeyOnlyFilter extends FilterBase {
|
|||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 0,
|
||||
"Expected: 0 but got: %s", filterArguments.size());
|
||||
return new KeyOnlyFilter();
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeBoolean(this.lenAsVal);
|
||||
}
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.io.DataInput;
|
|||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.TreeSet;
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* This filter is used for selecting only those keys with columns that matches
|
||||
|
@ -92,6 +93,15 @@ public class MultipleColumnPrefixFilter extends FilterBase {
|
|||
}
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
byte [][] prefixes = new byte [filterArguments.size()][];
|
||||
for (int i = 0 ; i < filterArguments.size(); i++) {
|
||||
byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(i));
|
||||
prefixes[i] = columnPrefix;
|
||||
}
|
||||
return new MultipleColumnPrefixFilter(prefixes);
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
out.writeInt(sortedPrefixes.size());
|
||||
for (byte [] element : sortedPrefixes) {
|
||||
|
|
|
@ -25,7 +25,9 @@ import java.io.DataInput;
|
|||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
/**
|
||||
* Implementation of Filter interface that limits results to a specific page
|
||||
* size. It terminates scanning once the number of filter-passed rows is >
|
||||
|
@ -55,6 +57,7 @@ public class PageFilter extends FilterBase {
|
|||
* @param pageSize Maximum result size.
|
||||
*/
|
||||
public PageFilter(final long pageSize) {
|
||||
Preconditions.checkArgument(pageSize >= 0, "must be positive %s", pageSize);
|
||||
this.pageSize = pageSize;
|
||||
}
|
||||
|
||||
|
@ -71,6 +74,13 @@ public class PageFilter extends FilterBase {
|
|||
return this.rowsAccepted > this.pageSize;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
long pageSize = ParseFilter.convertByteArrayToLong(filterArguments.get(0));
|
||||
return new PageFilter(pageSize);
|
||||
}
|
||||
|
||||
public void readFields(final DataInput in) throws IOException {
|
||||
this.pageSize = in.readLong();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,263 @@
|
|||
/**
|
||||
* Copyright 2011 The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import org.apache.hadoop.hbase.filter.*;
|
||||
|
||||
/**
|
||||
* ParseConstants holds a bunch of constants related to parsing Filter Strings
|
||||
* Used by {@link ParseFilter}
|
||||
*/
|
||||
public final class ParseConstants {
|
||||
|
||||
/**
|
||||
* ASCII code for LPAREN
|
||||
*/
|
||||
public static final int LPAREN = '(';
|
||||
|
||||
/**
|
||||
* ASCII code for RPAREN
|
||||
*/
|
||||
public static final int RPAREN = ')';
|
||||
|
||||
/**
|
||||
* ASCII code for whitespace
|
||||
*/
|
||||
public static final int WHITESPACE = ' ';
|
||||
|
||||
/**
|
||||
* ASCII code for tab
|
||||
*/
|
||||
public static final int TAB = '\t';
|
||||
|
||||
/**
|
||||
* ASCII code for 'A'
|
||||
*/
|
||||
public static final int A = 'A';
|
||||
|
||||
/**
|
||||
* ASCII code for 'N'
|
||||
*/
|
||||
public static final int N = 'N';
|
||||
|
||||
/**
|
||||
* ASCII code for 'D'
|
||||
*/
|
||||
public static final int D = 'D';
|
||||
|
||||
/**
|
||||
* ASCII code for 'O'
|
||||
*/
|
||||
public static final int O = 'O';
|
||||
|
||||
/**
|
||||
* ASCII code for 'R'
|
||||
*/
|
||||
public static final int R = 'R';
|
||||
|
||||
/**
|
||||
* ASCII code for 'S'
|
||||
*/
|
||||
public static final int S = 'S';
|
||||
|
||||
/**
|
||||
* ASCII code for 'K'
|
||||
*/
|
||||
public static final int K = 'K';
|
||||
|
||||
/**
|
||||
* ASCII code for 'I'
|
||||
*/
|
||||
public static final int I = 'I';
|
||||
|
||||
/**
|
||||
* ASCII code for 'P'
|
||||
*/
|
||||
public static final int P = 'P';
|
||||
|
||||
/**
|
||||
* SKIP Array
|
||||
*/
|
||||
public static final byte [] SKIP_ARRAY = new byte [ ] {'S', 'K', 'I', 'P'};
|
||||
public static final ByteBuffer SKIP_BUFFER = ByteBuffer.wrap(SKIP_ARRAY);
|
||||
|
||||
/**
|
||||
* ASCII code for 'W'
|
||||
*/
|
||||
public static final int W = 'W';
|
||||
|
||||
/**
|
||||
* ASCII code for 'H'
|
||||
*/
|
||||
public static final int H = 'H';
|
||||
|
||||
/**
|
||||
* ASCII code for 'L'
|
||||
*/
|
||||
public static final int L = 'L';
|
||||
|
||||
/**
|
||||
* ASCII code for 'E'
|
||||
*/
|
||||
public static final int E = 'E';
|
||||
|
||||
/**
|
||||
* WHILE Array
|
||||
*/
|
||||
public static final byte [] WHILE_ARRAY = new byte [] {'W', 'H', 'I', 'L', 'E'};
|
||||
public static final ByteBuffer WHILE_BUFFER = ByteBuffer.wrap(WHILE_ARRAY);
|
||||
|
||||
/**
|
||||
* OR Array
|
||||
*/
|
||||
public static final byte [] OR_ARRAY = new byte [] {'O','R'};
|
||||
public static final ByteBuffer OR_BUFFER = ByteBuffer.wrap(OR_ARRAY);
|
||||
|
||||
/**
|
||||
* AND Array
|
||||
*/
|
||||
public static final byte [] AND_ARRAY = new byte [] {'A','N', 'D'};
|
||||
public static final ByteBuffer AND_BUFFER = ByteBuffer.wrap(AND_ARRAY);
|
||||
|
||||
/**
|
||||
* ASCII code for Backslash
|
||||
*/
|
||||
public static final int BACKSLASH = '\\';
|
||||
|
||||
/**
|
||||
* ASCII code for a single quote
|
||||
*/
|
||||
public static final int SINGLE_QUOTE = '\'';
|
||||
|
||||
/**
|
||||
* ASCII code for a comma
|
||||
*/
|
||||
public static final int COMMA = ',';
|
||||
|
||||
/**
|
||||
* LESS_THAN Array
|
||||
*/
|
||||
public static final byte [] LESS_THAN_ARRAY = new byte [] {'<'};
|
||||
public static final ByteBuffer LESS_THAN_BUFFER = ByteBuffer.wrap(LESS_THAN_ARRAY);
|
||||
|
||||
/**
|
||||
* LESS_THAN_OR_EQUAL_TO Array
|
||||
*/
|
||||
public static final byte [] LESS_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'<', '='};
|
||||
public static final ByteBuffer LESS_THAN_OR_EQUAL_TO_BUFFER =
|
||||
ByteBuffer.wrap(LESS_THAN_OR_EQUAL_TO_ARRAY);
|
||||
|
||||
/**
|
||||
* GREATER_THAN Array
|
||||
*/
|
||||
public static final byte [] GREATER_THAN_ARRAY = new byte [] {'>'};
|
||||
public static final ByteBuffer GREATER_THAN_BUFFER = ByteBuffer.wrap(GREATER_THAN_ARRAY);
|
||||
|
||||
/**
|
||||
* GREATER_THAN_OR_EQUAL_TO Array
|
||||
*/
|
||||
public static final byte [] GREATER_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'>', '='};
|
||||
public static final ByteBuffer GREATER_THAN_OR_EQUAL_TO_BUFFER =
|
||||
ByteBuffer.wrap(GREATER_THAN_OR_EQUAL_TO_ARRAY);
|
||||
|
||||
/**
|
||||
* EQUAL_TO Array
|
||||
*/
|
||||
public static final byte [] EQUAL_TO_ARRAY = new byte [] {'='};
|
||||
public static final ByteBuffer EQUAL_TO_BUFFER = ByteBuffer.wrap(EQUAL_TO_ARRAY);
|
||||
|
||||
/**
|
||||
* NOT_EQUAL_TO Array
|
||||
*/
|
||||
public static final byte [] NOT_EQUAL_TO_ARRAY = new byte [] {'!', '='};
|
||||
public static final ByteBuffer NOT_EQUAL_TO_BUFFER = ByteBuffer.wrap(NOT_EQUAL_TO_ARRAY);
|
||||
|
||||
/**
|
||||
* ASCII code for equal to (=)
|
||||
*/
|
||||
public static final int EQUAL_TO = '=';
|
||||
|
||||
/**
|
||||
* AND Byte Array
|
||||
*/
|
||||
public static final byte [] AND = new byte [] {'A','N','D'};
|
||||
|
||||
/**
|
||||
* OR Byte Array
|
||||
*/
|
||||
public static final byte [] OR = new byte [] {'O', 'R'};
|
||||
|
||||
/**
|
||||
* LPAREN Array
|
||||
*/
|
||||
public static final byte [] LPAREN_ARRAY = new byte [] {'('};
|
||||
public static final ByteBuffer LPAREN_BUFFER = ByteBuffer.wrap(LPAREN_ARRAY);
|
||||
|
||||
/**
|
||||
* ASCII code for colon (:)
|
||||
*/
|
||||
public static final int COLON = ':';
|
||||
|
||||
/**
|
||||
* ASCII code for Zero
|
||||
*/
|
||||
public static final int ZERO = '0';
|
||||
|
||||
/**
|
||||
* ASCII code foe Nine
|
||||
*/
|
||||
public static final int NINE = '9';
|
||||
|
||||
/**
|
||||
* BinaryType byte array
|
||||
*/
|
||||
public static final byte [] binaryType = new byte [] {'b','i','n','a','r','y'};
|
||||
|
||||
/**
|
||||
* BinaryPrefixType byte array
|
||||
*/
|
||||
public static final byte [] binaryPrefixType = new byte [] {'b','i','n','a','r','y',
|
||||
'p','r','e','f','i','x'};
|
||||
|
||||
/**
|
||||
* RegexStringType byte array
|
||||
*/
|
||||
public static final byte [] regexStringType = new byte [] {'r','e','g','e', 'x',
|
||||
's','t','r','i','n','g'};
|
||||
|
||||
/**
|
||||
* SubstringType byte array
|
||||
*/
|
||||
public static final byte [] substringType = new byte [] {'s','u','b','s','t','r','i','n','g'};
|
||||
|
||||
/**
|
||||
* ASCII for Minus Sign
|
||||
*/
|
||||
public static final int MINUS_SIGN = '-';
|
||||
|
||||
/**
|
||||
* Package containing filters
|
||||
*/
|
||||
public static final String FILTER_PACKAGE = "org.apache.hadoop.hbase.filter";
|
||||
}
|
|
@ -0,0 +1,796 @@
|
|||
/**
|
||||
* 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.lang.reflect.*;
|
||||
import java.util.EmptyStackException;
|
||||
|
||||
/**
|
||||
* This class allows a user to specify a filter via a string
|
||||
* The string is parsed using the methods of this class and
|
||||
* a filter object is constructed. This filter object is then wrapped
|
||||
* in a scanner object which is then returned
|
||||
* <p>
|
||||
* This class addresses the HBASE-4168 JIRA. More documentaton on this
|
||||
* Filter Language can be found at: https://issues.apache.org/jira/browse/HBASE-4176
|
||||
*/
|
||||
public class ParseFilter {
|
||||
|
||||
private HashMap<ByteBuffer, Integer> operatorPrecedenceHashMap;
|
||||
|
||||
/**
|
||||
* Constructor
|
||||
* <p>
|
||||
* Creates the operatorPrecedenceHashMap
|
||||
*/
|
||||
public ParseFilter() {
|
||||
operatorPrecedenceHashMap = new HashMap<ByteBuffer, Integer>();
|
||||
operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
|
||||
operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
|
||||
operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
|
||||
operatorPrecedenceHashMap.put(ParseConstants.OR_BUFFER, 3);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the filterString and constructs a filter using it
|
||||
* <p>
|
||||
* @param filterString filter string given by the user
|
||||
* @return filter object we constructed
|
||||
*/
|
||||
public Filter parseFilterString (String filterString)
|
||||
throws CharacterCodingException {
|
||||
return parseFilterString(Bytes.toBytes(filterString));
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the filterString and constructs a filter using it
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @return filter object we constructed
|
||||
*/
|
||||
public Filter parseFilterString (byte [] filterStringAsByteArray)
|
||||
throws CharacterCodingException {
|
||||
// stack for the operators and parenthesis
|
||||
Stack <ByteBuffer> operatorStack = new Stack<ByteBuffer>();
|
||||
// stack for the filter objects
|
||||
Stack <Filter> filterStack = new Stack<Filter>();
|
||||
|
||||
Filter filter = null;
|
||||
for (int i=0; i<filterStringAsByteArray.length; i++) {
|
||||
if (filterStringAsByteArray[i] == ParseConstants.LPAREN) {
|
||||
// LPAREN found
|
||||
operatorStack.push(ParseConstants.LPAREN_BUFFER);
|
||||
} else if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[i] == ParseConstants.TAB) {
|
||||
// WHITESPACE or TAB found
|
||||
continue;
|
||||
} else if (checkForOr(filterStringAsByteArray, i)) {
|
||||
// OR found
|
||||
i += ParseConstants.OR_ARRAY.length - 1;
|
||||
reduce(operatorStack, filterStack, ParseConstants.OR_BUFFER);
|
||||
operatorStack.push(ParseConstants.OR_BUFFER);
|
||||
} else if (checkForAnd(filterStringAsByteArray, i)) {
|
||||
// AND found
|
||||
i += ParseConstants.AND_ARRAY.length - 1;
|
||||
reduce(operatorStack, filterStack, ParseConstants.AND_BUFFER);
|
||||
operatorStack.push(ParseConstants.AND_BUFFER);
|
||||
} else if (checkForSkip(filterStringAsByteArray, i)) {
|
||||
// SKIP found
|
||||
i += ParseConstants.SKIP_ARRAY.length - 1;
|
||||
reduce(operatorStack, filterStack, ParseConstants.SKIP_BUFFER);
|
||||
operatorStack.push(ParseConstants.SKIP_BUFFER);
|
||||
} else if (checkForWhile(filterStringAsByteArray, i)) {
|
||||
// WHILE found
|
||||
i += ParseConstants.WHILE_ARRAY.length - 1;
|
||||
reduce(operatorStack, filterStack, ParseConstants.WHILE_BUFFER);
|
||||
operatorStack.push(ParseConstants.WHILE_BUFFER);
|
||||
} else if (filterStringAsByteArray[i] == ParseConstants.RPAREN) {
|
||||
// RPAREN found
|
||||
if (operatorStack.empty()) {
|
||||
throw new IllegalArgumentException("Mismatched parenthesis");
|
||||
}
|
||||
ByteBuffer argumentOnTopOfStack = operatorStack.peek();
|
||||
while (!(argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER))) {
|
||||
filterStack.push(popArguments(operatorStack, filterStack));
|
||||
if (operatorStack.empty()) {
|
||||
throw new IllegalArgumentException("Mismatched parenthesis");
|
||||
}
|
||||
argumentOnTopOfStack = operatorStack.pop();
|
||||
}
|
||||
} else {
|
||||
// SimpleFilterExpression found
|
||||
byte [] filterSimpleExpression = extractFilterSimpleExpression(filterStringAsByteArray, i);
|
||||
i+= (filterSimpleExpression.length - 1);
|
||||
filter = parseSimpleFilterExpression(filterSimpleExpression);
|
||||
filterStack.push(filter);
|
||||
}
|
||||
}
|
||||
|
||||
// Finished parsing filterString
|
||||
while (!operatorStack.empty()) {
|
||||
filterStack.push(popArguments(operatorStack, filterStack));
|
||||
}
|
||||
filter = filterStack.pop();
|
||||
if (!filterStack.empty()) {
|
||||
throw new IllegalArgumentException("Incorrect Filter String");
|
||||
}
|
||||
return filter;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts a simple filter expression from the filter string given by the user
|
||||
* <p>
|
||||
* A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg')
|
||||
* The user given filter string can have many simpleFilterExpressions combined
|
||||
* using operators.
|
||||
* <p>
|
||||
* This function extracts a simpleFilterExpression from the
|
||||
* larger filterString given the start offset of the simpler expression
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @param filterExpressionStartOffset start index of the simple filter expression
|
||||
* @return byte array containing the simple filter expression
|
||||
*/
|
||||
public byte [] extractFilterSimpleExpression (byte [] filterStringAsByteArray,
|
||||
int filterExpressionStartOffset)
|
||||
throws CharacterCodingException {
|
||||
int quoteCount = 0;
|
||||
for (int i=filterExpressionStartOffset; i<filterStringAsByteArray.length; i++) {
|
||||
if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
|
||||
if (isQuoteUnescaped(filterStringAsByteArray, i)) {
|
||||
quoteCount ++;
|
||||
} else {
|
||||
// To skip the next quote that has been escaped
|
||||
i++;
|
||||
}
|
||||
}
|
||||
if (filterStringAsByteArray[i] == ParseConstants.RPAREN && (quoteCount %2 ) == 0) {
|
||||
byte [] filterSimpleExpression = new byte [i - filterExpressionStartOffset + 1];
|
||||
Bytes.putBytes(filterSimpleExpression, 0, filterStringAsByteArray,
|
||||
filterExpressionStartOffset, i-filterExpressionStartOffset + 1);
|
||||
return filterSimpleExpression;
|
||||
}
|
||||
}
|
||||
throw new IllegalArgumentException("Incorrect Filter String");
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a filter object given a simple filter expression
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @return filter object we constructed
|
||||
*/
|
||||
public Filter parseSimpleFilterExpression (byte [] filterStringAsByteArray)
|
||||
throws CharacterCodingException {
|
||||
|
||||
String filterName = Bytes.toString(getFilterName(filterStringAsByteArray));
|
||||
ArrayList<byte []> filterArguments = getFilterArguments(filterStringAsByteArray);
|
||||
try {
|
||||
filterName = ParseConstants.FILTER_PACKAGE + "." + filterName;
|
||||
Class c = Class.forName(filterName);
|
||||
Class[] argTypes = new Class [] {ArrayList.class};
|
||||
Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
|
||||
return (Filter) m.invoke(null,filterArguments);
|
||||
} catch (ClassNotFoundException e) {
|
||||
e.printStackTrace();
|
||||
} catch (NoSuchMethodException e) {
|
||||
e.printStackTrace();
|
||||
} catch (IllegalAccessException e) {
|
||||
e.printStackTrace();
|
||||
} catch (InvocationTargetException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
throw new IllegalArgumentException("Incorrect filter string " +
|
||||
new String(filterStringAsByteArray));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the filter name given a simple filter expression
|
||||
* <p>
|
||||
* @param filterStringAsByteArray a simple filter expression
|
||||
* @return name of filter in the simple filter expression
|
||||
*/
|
||||
public static byte [] getFilterName (byte [] filterStringAsByteArray) {
|
||||
int filterNameStartIndex = 0;
|
||||
int filterNameEndIndex = 0;
|
||||
|
||||
for (int i=filterNameStartIndex; i<filterStringAsByteArray.length; i++) {
|
||||
if (filterStringAsByteArray[i] == ParseConstants.LPAREN ||
|
||||
filterStringAsByteArray[i] == ParseConstants.WHITESPACE) {
|
||||
filterNameEndIndex = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (filterNameEndIndex == 0) {
|
||||
throw new IllegalArgumentException("Incorrect Filter Name");
|
||||
}
|
||||
|
||||
byte [] filterName = new byte[filterNameEndIndex - filterNameStartIndex];
|
||||
Bytes.putBytes(filterName, 0, filterStringAsByteArray, 0,
|
||||
filterNameEndIndex - filterNameStartIndex);
|
||||
return filterName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the arguments of the filter from the filter string
|
||||
* <p>
|
||||
* @param filter_string filter string given by the user
|
||||
* @return an ArrayList containing the arguments of the filter in the filter string
|
||||
*/
|
||||
public static ArrayList<byte []> getFilterArguments (byte [] filterStringAsByteArray) {
|
||||
int argumentListStartIndex = KeyValue.getDelimiter(filterStringAsByteArray, 0,
|
||||
filterStringAsByteArray.length,
|
||||
ParseConstants.LPAREN);
|
||||
if (argumentListStartIndex == -1) {
|
||||
throw new IllegalArgumentException("Incorrect argument list");
|
||||
}
|
||||
|
||||
int argumentStartIndex = 0;
|
||||
int argumentEndIndex = 0;
|
||||
ArrayList<byte []> filterArguments = new ArrayList<byte []>();
|
||||
|
||||
for (int i = argumentListStartIndex + 1; i<filterStringAsByteArray.length; i++) {
|
||||
|
||||
if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[i] == ParseConstants.COMMA ||
|
||||
filterStringAsByteArray[i] == ParseConstants.RPAREN) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// The argument is in single quotes - for example 'prefix'
|
||||
if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
|
||||
argumentStartIndex = i;
|
||||
for (int j = argumentStartIndex+1; j < filterStringAsByteArray.length; j++) {
|
||||
if (filterStringAsByteArray[j] == ParseConstants.SINGLE_QUOTE) {
|
||||
if (isQuoteUnescaped(filterStringAsByteArray,j)) {
|
||||
argumentEndIndex = j;
|
||||
i = j+1;
|
||||
byte [] filterArgument = createUnescapdArgument(filterStringAsByteArray,
|
||||
argumentStartIndex, argumentEndIndex);
|
||||
filterArguments.add(filterArgument);
|
||||
break;
|
||||
} else {
|
||||
// To jump over the second escaped quote
|
||||
j++;
|
||||
}
|
||||
} else if (j == filterStringAsByteArray.length - 1) {
|
||||
throw new IllegalArgumentException("Incorrect argument list");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// The argument is an integer, boolean, comparison operator like <, >, != etc
|
||||
argumentStartIndex = i;
|
||||
for (int j = argumentStartIndex; j < filterStringAsByteArray.length; j++) {
|
||||
if (filterStringAsByteArray[j] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[j] == ParseConstants.COMMA ||
|
||||
filterStringAsByteArray[j] == ParseConstants.RPAREN) {
|
||||
argumentEndIndex = j - 1;
|
||||
i = j;
|
||||
byte [] filterArgument = new byte [argumentEndIndex - argumentStartIndex + 1];
|
||||
Bytes.putBytes(filterArgument, 0, filterStringAsByteArray,
|
||||
argumentStartIndex, argumentEndIndex - argumentStartIndex + 1);
|
||||
filterArguments.add(filterArgument);
|
||||
break;
|
||||
} else if (j == filterStringAsByteArray.length - 1) {
|
||||
throw new IllegalArgumentException("Incorrect argument list");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return filterArguments;
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is called while parsing the filterString and an operator is parsed
|
||||
* <p>
|
||||
* @param operatorStack the stack containing the operators and parenthesis
|
||||
* @param filterStack the stack containing the filters
|
||||
* @param operator the operator found while parsing the filterString
|
||||
* @return returns the filterStack after evaluating the stack
|
||||
*/
|
||||
public void reduce(Stack<ByteBuffer> operatorStack,
|
||||
Stack<Filter> filterStack,
|
||||
ByteBuffer operator) {
|
||||
while (!operatorStack.empty() &&
|
||||
!(ParseConstants.LPAREN_BUFFER.equals(operatorStack.peek())) &&
|
||||
hasHigherPriority(operatorStack.peek(), operator)) {
|
||||
filterStack.push(popArguments(operatorStack, filterStack));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Pops an argument from the operator stack and the number of arguments required by the operator
|
||||
* from the filterStack and evaluates them
|
||||
* <p>
|
||||
* @param operatorStack the stack containing the operators
|
||||
* @param filterStack the stack containing the filters
|
||||
* @return the evaluated filter
|
||||
*/
|
||||
public static Filter popArguments (Stack<ByteBuffer> operatorStack, Stack <Filter> filterStack) {
|
||||
ByteBuffer argumentOnTopOfStack = operatorStack.peek();
|
||||
|
||||
if (argumentOnTopOfStack.equals(ParseConstants.OR_BUFFER)) {
|
||||
// The top of the stack is an OR
|
||||
try {
|
||||
ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
|
||||
while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.OR_BUFFER)) {
|
||||
Filter filter = filterStack.pop();
|
||||
listOfFilters.add(0, filter);
|
||||
operatorStack.pop();
|
||||
}
|
||||
Filter filter = filterStack.pop();
|
||||
listOfFilters.add(0, filter);
|
||||
Filter orFilter = new FilterList(FilterList.Operator.MUST_PASS_ONE, listOfFilters);
|
||||
return orFilter;
|
||||
} catch (EmptyStackException e) {
|
||||
throw new IllegalArgumentException("Incorrect input string - an OR needs two filters");
|
||||
}
|
||||
|
||||
} else if (argumentOnTopOfStack.equals(ParseConstants.AND_BUFFER)) {
|
||||
// The top of the stack is an AND
|
||||
try {
|
||||
ArrayList<Filter> listOfFilters = new ArrayList<Filter>();
|
||||
while (!operatorStack.empty() && operatorStack.peek().equals(ParseConstants.AND_BUFFER)) {
|
||||
Filter filter = filterStack.pop();
|
||||
listOfFilters.add(0, filter);
|
||||
operatorStack.pop();
|
||||
}
|
||||
Filter filter = filterStack.pop();
|
||||
listOfFilters.add(0, filter);
|
||||
Filter andFilter = new FilterList(FilterList.Operator.MUST_PASS_ALL, listOfFilters);
|
||||
return andFilter;
|
||||
} catch (EmptyStackException e) {
|
||||
throw new IllegalArgumentException("Incorrect input string - an AND needs two filters");
|
||||
}
|
||||
|
||||
} else if (argumentOnTopOfStack.equals(ParseConstants.SKIP_BUFFER)) {
|
||||
// The top of the stack is a SKIP
|
||||
try {
|
||||
Filter wrappedFilter = filterStack.pop();
|
||||
Filter skipFilter = new SkipFilter(wrappedFilter);
|
||||
operatorStack.pop();
|
||||
return skipFilter;
|
||||
} catch (EmptyStackException e) {
|
||||
throw new IllegalArgumentException("Incorrect input string - a SKIP wraps a filter");
|
||||
}
|
||||
|
||||
} else if (argumentOnTopOfStack.equals(ParseConstants.WHILE_BUFFER)) {
|
||||
// The top of the stack is a WHILE
|
||||
try {
|
||||
Filter wrappedFilter = filterStack.pop();
|
||||
Filter whileMatchFilter = new WhileMatchFilter(wrappedFilter);
|
||||
operatorStack.pop();
|
||||
return whileMatchFilter;
|
||||
} catch (EmptyStackException e) {
|
||||
throw new IllegalArgumentException("Incorrect input string - a WHILE wraps a filter");
|
||||
}
|
||||
|
||||
} else if (argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER)) {
|
||||
// The top of the stack is a LPAREN
|
||||
try {
|
||||
Filter filter = filterStack.pop();
|
||||
operatorStack.pop();
|
||||
return filter;
|
||||
} catch (EmptyStackException e) {
|
||||
throw new IllegalArgumentException("Incorrect Filter String");
|
||||
}
|
||||
|
||||
} else {
|
||||
throw new IllegalArgumentException("Incorrect arguments on operatorStack");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns which operator has higher precedence
|
||||
* <p>
|
||||
* If a has higher precedence than b, it returns true
|
||||
* If they have the same precedence, it returns false
|
||||
*/
|
||||
public boolean hasHigherPriority(ByteBuffer a, ByteBuffer b) {
|
||||
if ((operatorPrecedenceHashMap.get(a) - operatorPrecedenceHashMap.get(b)) < 0) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the single quote escaping a single quote - thus it returns an unescaped argument
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by user
|
||||
* @param argumentStartIndex start index of the argument
|
||||
* @param argumentEndIndex end index of the argument
|
||||
* @return returns an unescaped argument
|
||||
*/
|
||||
public static byte [] createUnescapdArgument (byte [] filterStringAsByteArray,
|
||||
int argumentStartIndex, int argumentEndIndex) {
|
||||
int unescapedArgumentLength = 2;
|
||||
for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
|
||||
unescapedArgumentLength ++;
|
||||
if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE &&
|
||||
i != (argumentEndIndex - 1) &&
|
||||
filterStringAsByteArray[i+1] == ParseConstants.SINGLE_QUOTE) {
|
||||
i++;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
byte [] unescapedArgument = new byte [unescapedArgumentLength];
|
||||
int count = 1;
|
||||
unescapedArgument[0] = '\'';
|
||||
for (int i = argumentStartIndex + 1; i <= argumentEndIndex - 1; i++) {
|
||||
if (filterStringAsByteArray [i] == ParseConstants.SINGLE_QUOTE &&
|
||||
i != (argumentEndIndex - 1) &&
|
||||
filterStringAsByteArray [i+1] == ParseConstants.SINGLE_QUOTE) {
|
||||
unescapedArgument[count++] = filterStringAsByteArray [i+1];
|
||||
i++;
|
||||
}
|
||||
else {
|
||||
unescapedArgument[count++] = filterStringAsByteArray [i];
|
||||
}
|
||||
}
|
||||
unescapedArgument[unescapedArgumentLength - 1] = '\'';
|
||||
return unescapedArgument;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the current index of filter string we are on is the beginning of the keyword 'OR'
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @param indexOfOr index at which an 'O' was read
|
||||
* @return true if the keyword 'OR' is at the current index
|
||||
*/
|
||||
public static boolean checkForOr (byte [] filterStringAsByteArray, int indexOfOr)
|
||||
throws CharacterCodingException, ArrayIndexOutOfBoundsException {
|
||||
|
||||
try {
|
||||
if (filterStringAsByteArray[indexOfOr] == ParseConstants.O &&
|
||||
filterStringAsByteArray[indexOfOr+1] == ParseConstants.R &&
|
||||
(filterStringAsByteArray[indexOfOr-1] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfOr-1] == ParseConstants.RPAREN) &&
|
||||
(filterStringAsByteArray[indexOfOr+2] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfOr+2] == ParseConstants.LPAREN)) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the current index of filter string we are on is the beginning of the keyword 'AND'
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @param indexOfAnd index at which an 'A' was read
|
||||
* @return true if the keyword 'AND' is at the current index
|
||||
*/
|
||||
public static boolean checkForAnd (byte [] filterStringAsByteArray, int indexOfAnd)
|
||||
throws CharacterCodingException {
|
||||
|
||||
try {
|
||||
if (filterStringAsByteArray[indexOfAnd] == ParseConstants.A &&
|
||||
filterStringAsByteArray[indexOfAnd+1] == ParseConstants.N &&
|
||||
filterStringAsByteArray[indexOfAnd+2] == ParseConstants.D &&
|
||||
(filterStringAsByteArray[indexOfAnd-1] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfAnd-1] == ParseConstants.RPAREN) &&
|
||||
(filterStringAsByteArray[indexOfAnd+3] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfAnd+3] == ParseConstants.LPAREN)) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the current index of filter string we are on is the beginning of the keyword 'SKIP'
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @param indexOfSkip index at which an 'S' was read
|
||||
* @return true if the keyword 'SKIP' is at the current index
|
||||
*/
|
||||
public static boolean checkForSkip (byte [] filterStringAsByteArray, int indexOfSkip)
|
||||
throws CharacterCodingException {
|
||||
|
||||
try {
|
||||
if (filterStringAsByteArray[indexOfSkip] == ParseConstants.S &&
|
||||
filterStringAsByteArray[indexOfSkip+1] == ParseConstants.K &&
|
||||
filterStringAsByteArray[indexOfSkip+2] == ParseConstants.I &&
|
||||
filterStringAsByteArray[indexOfSkip+3] == ParseConstants.P &&
|
||||
(indexOfSkip == 0 ||
|
||||
filterStringAsByteArray[indexOfSkip-1] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfSkip-1] == ParseConstants.RPAREN ||
|
||||
filterStringAsByteArray[indexOfSkip-1] == ParseConstants.LPAREN) &&
|
||||
(filterStringAsByteArray[indexOfSkip+4] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfSkip+4] == ParseConstants.LPAREN)) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the current index of filter string we are on is the beginning of the keyword 'WHILE'
|
||||
* <p>
|
||||
* @param filterStringAsByteArray filter string given by the user
|
||||
* @param indexOfWhile index at which an 'W' was read
|
||||
* @return true if the keyword 'WHILE' is at the current index
|
||||
*/
|
||||
public static boolean checkForWhile (byte [] filterStringAsByteArray, int indexOfWhile)
|
||||
throws CharacterCodingException {
|
||||
|
||||
try {
|
||||
if (filterStringAsByteArray[indexOfWhile] == ParseConstants.W &&
|
||||
filterStringAsByteArray[indexOfWhile+1] == ParseConstants.H &&
|
||||
filterStringAsByteArray[indexOfWhile+2] == ParseConstants.I &&
|
||||
filterStringAsByteArray[indexOfWhile+3] == ParseConstants.L &&
|
||||
filterStringAsByteArray[indexOfWhile+4] == ParseConstants.E &&
|
||||
(indexOfWhile == 0 || filterStringAsByteArray[indexOfWhile-1] == ParseConstants.WHITESPACE
|
||||
|| filterStringAsByteArray[indexOfWhile-1] == ParseConstants.RPAREN ||
|
||||
filterStringAsByteArray[indexOfWhile-1] == ParseConstants.LPAREN) &&
|
||||
(filterStringAsByteArray[indexOfWhile+5] == ParseConstants.WHITESPACE ||
|
||||
filterStringAsByteArray[indexOfWhile+5] == ParseConstants.LPAREN)) {
|
||||
return true;
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
} catch (ArrayIndexOutOfBoundsException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a boolean indicating whether the quote was escaped or not
|
||||
* <p>
|
||||
* @param array byte array in which the quote was found
|
||||
* @param quoteIndex index of the single quote
|
||||
* @return returns true if the quote was unescaped
|
||||
*/
|
||||
public static boolean isQuoteUnescaped (byte [] array, int quoteIndex) {
|
||||
if (array == null) {
|
||||
throw new IllegalArgumentException("isQuoteUnescaped called with a null array");
|
||||
}
|
||||
|
||||
if (quoteIndex == array.length - 1 || array[quoteIndex+1] != ParseConstants.SINGLE_QUOTE) {
|
||||
return true;
|
||||
}
|
||||
else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Takes a quoted byte array and converts it into an unquoted byte array
|
||||
* For example: given a byte array representing 'abc', it returns a
|
||||
* byte array representing abc
|
||||
* <p>
|
||||
* @param quotedByteArray the quoted byte array
|
||||
* @return
|
||||
*/
|
||||
public static byte [] removeQuotesFromByteArray (byte [] quotedByteArray) {
|
||||
if (quotedByteArray == null ||
|
||||
quotedByteArray.length < 2 ||
|
||||
quotedByteArray[0] != ParseConstants.SINGLE_QUOTE ||
|
||||
quotedByteArray[quotedByteArray.length - 1] != ParseConstants.SINGLE_QUOTE) {
|
||||
throw new IllegalArgumentException("removeQuotesFromByteArray needs a quoted byte array");
|
||||
} else {
|
||||
byte [] targetString = new byte [quotedByteArray.length - 2];
|
||||
Bytes.putBytes(targetString, 0, quotedByteArray, 1, quotedByteArray.length - 2);
|
||||
return targetString;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts an int expressed in a byte array to an actual int
|
||||
* <p>
|
||||
* This doesn't use Bytes.toInt because that assumes
|
||||
* that there will be {@link #SIZEOF_INT} bytes available.
|
||||
* <p>
|
||||
* @param numberAsByteArray the int value expressed as a byte array
|
||||
* @return the int value
|
||||
*/
|
||||
public static int convertByteArrayToInt (byte [] numberAsByteArray) {
|
||||
|
||||
long tempResult = ParseFilter.convertByteArrayToLong(numberAsByteArray);
|
||||
|
||||
if (tempResult > Integer.MAX_VALUE) {
|
||||
throw new IllegalArgumentException("Integer Argument too large");
|
||||
} else if (tempResult < Integer.MIN_VALUE) {
|
||||
throw new IllegalArgumentException("Integer Argument too small");
|
||||
}
|
||||
|
||||
int result = (int) tempResult;
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a long expressed in a byte array to an actual long
|
||||
* <p>
|
||||
* This doesn't use Bytes.toLong because that assumes
|
||||
* that there will be {@link #SIZEOF_LONG} bytes available.
|
||||
* <p>
|
||||
* @param numberAsByteArray the long value expressed as a byte array
|
||||
* @return the long value
|
||||
*/
|
||||
public static long convertByteArrayToLong (byte [] numberAsByteArray) {
|
||||
if (numberAsByteArray == null) {
|
||||
throw new IllegalArgumentException("convertByteArrayToLong called with a null array");
|
||||
}
|
||||
|
||||
int i = 0;
|
||||
long result = 0;
|
||||
boolean isNegative = false;
|
||||
|
||||
if (numberAsByteArray[i] == ParseConstants.MINUS_SIGN) {
|
||||
i++;
|
||||
isNegative = true;
|
||||
}
|
||||
|
||||
while (i != numberAsByteArray.length) {
|
||||
if (numberAsByteArray[i] < ParseConstants.ZERO ||
|
||||
numberAsByteArray[i] > ParseConstants.NINE) {
|
||||
throw new IllegalArgumentException("Byte Array should only contain digits");
|
||||
}
|
||||
result = result*10 + (numberAsByteArray[i] - ParseConstants.ZERO);
|
||||
if (result < 0) {
|
||||
throw new IllegalArgumentException("Long Argument too large");
|
||||
}
|
||||
i++;
|
||||
}
|
||||
|
||||
if (isNegative) {
|
||||
return -result;
|
||||
} else {
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts a boolean expressed in a byte array to an actual boolean
|
||||
*<p>
|
||||
* This doesn't used Bytes.toBoolean because Bytes.toBoolean(byte [])
|
||||
* assumes that 1 stands for true and 0 for false.
|
||||
* Here, the byte array representing "true" and "false" is parsed
|
||||
* <p>
|
||||
* @param booleanAsByteArray the boolean value expressed as a byte array
|
||||
* @return the boolean value
|
||||
*/
|
||||
public static boolean convertByteArrayToBoolean (byte [] booleanAsByteArray) {
|
||||
if (booleanAsByteArray == null) {
|
||||
throw new IllegalArgumentException("convertByteArrayToBoolean called with a null array");
|
||||
}
|
||||
|
||||
if (booleanAsByteArray.length == 4 &&
|
||||
(booleanAsByteArray[0] == 't' || booleanAsByteArray[0] == 'T') &&
|
||||
(booleanAsByteArray[1] == 'r' || booleanAsByteArray[1] == 'R') &&
|
||||
(booleanAsByteArray[2] == 'u' || booleanAsByteArray[2] == 'U') &&
|
||||
(booleanAsByteArray[3] == 'e' || booleanAsByteArray[3] == 'E')) {
|
||||
return true;
|
||||
}
|
||||
else if (booleanAsByteArray.length == 5 &&
|
||||
(booleanAsByteArray[0] == 'f' || booleanAsByteArray[0] == 'F') &&
|
||||
(booleanAsByteArray[1] == 'a' || booleanAsByteArray[1] == 'A') &&
|
||||
(booleanAsByteArray[2] == 'l' || booleanAsByteArray[2] == 'L') &&
|
||||
(booleanAsByteArray[3] == 's' || booleanAsByteArray[3] == 'S') &&
|
||||
(booleanAsByteArray[4] == 'e' || booleanAsByteArray[4] == 'E')) {
|
||||
return false;
|
||||
}
|
||||
else {
|
||||
throw new IllegalArgumentException("Incorrect Boolean Expression");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Takes a compareOperator symbol as a byte array and returns the corresponding CompareOperator
|
||||
* <p>
|
||||
* @param compareOpAsByteArray the comparatorOperator symbol as a byte array
|
||||
* @return the Compare Operator
|
||||
*/
|
||||
public static CompareFilter.CompareOp createCompareOp (byte [] compareOpAsByteArray) {
|
||||
ByteBuffer compareOp = ByteBuffer.wrap(compareOpAsByteArray);
|
||||
if (compareOp.equals(ParseConstants.LESS_THAN_BUFFER))
|
||||
return CompareOp.LESS;
|
||||
else if (compareOp.equals(ParseConstants.LESS_THAN_OR_EQUAL_TO_BUFFER))
|
||||
return CompareOp.LESS_OR_EQUAL;
|
||||
else if (compareOp.equals(ParseConstants.GREATER_THAN_BUFFER))
|
||||
return CompareOp.GREATER;
|
||||
else if (compareOp.equals(ParseConstants.GREATER_THAN_OR_EQUAL_TO_BUFFER))
|
||||
return CompareOp.GREATER_OR_EQUAL;
|
||||
else if (compareOp.equals(ParseConstants.NOT_EQUAL_TO_BUFFER))
|
||||
return CompareOp.NOT_EQUAL;
|
||||
else if (compareOp.equals(ParseConstants.EQUAL_TO_BUFFER))
|
||||
return CompareOp.EQUAL;
|
||||
else
|
||||
throw new IllegalArgumentException("Invalid compare operator");
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses a comparator of the form comparatorType:comparatorValue form and returns a comparator
|
||||
* <p>
|
||||
* @param comparator the comparator in the form comparatorType:comparatorValue
|
||||
* @return the parsed comparator
|
||||
*/
|
||||
public static WritableByteArrayComparable createComparator (byte [] comparator) {
|
||||
if (comparator == null)
|
||||
throw new IllegalArgumentException("Incorrect Comparator");
|
||||
byte [][] parsedComparator = ParseFilter.parseComparator(comparator);
|
||||
byte [] comparatorType = parsedComparator[0];
|
||||
byte [] comparatorValue = parsedComparator[1];
|
||||
|
||||
|
||||
if (Bytes.equals(comparatorType, ParseConstants.binaryType))
|
||||
return new BinaryComparator(comparatorValue);
|
||||
else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
|
||||
return new BinaryPrefixComparator(comparatorValue);
|
||||
else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
|
||||
return new RegexStringComparator(new String(comparatorValue));
|
||||
else if (Bytes.equals(comparatorType, ParseConstants.substringType))
|
||||
return new SubstringComparator(new String(comparatorValue));
|
||||
else
|
||||
throw new IllegalArgumentException("Incorrect comparatorType");
|
||||
}
|
||||
|
||||
/**
|
||||
* Splits a column in comparatorType:comparatorValue form into separate byte arrays
|
||||
* <p>
|
||||
* @param comparator the comparator
|
||||
* @return the parsed arguments of the comparator as a 2D byte array
|
||||
*/
|
||||
public static byte [][] parseComparator (byte [] comparator) {
|
||||
final int index = KeyValue.getDelimiter(comparator, 0, comparator.length, ParseConstants.COLON);
|
||||
if (index == -1) {
|
||||
throw new IllegalArgumentException("Incorrect comparator");
|
||||
}
|
||||
|
||||
byte [][] result = new byte [2][0];
|
||||
result[0] = new byte [index];
|
||||
System.arraycopy(comparator, 0, result[0], 0, index);
|
||||
|
||||
final int len = comparator.length - (index + 1);
|
||||
result[1] = new byte[len];
|
||||
System.arraycopy(comparator, index + 1, result[1], 0, len);
|
||||
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -27,6 +27,9 @@ import java.io.DataOutput;
|
|||
import java.io.IOException;
|
||||
import java.io.DataInput;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Pass results that have same row prefix.
|
||||
|
@ -67,6 +70,13 @@ public class PrefixFilter extends FilterBase {
|
|||
return passedPrefix;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
Preconditions.checkArgument(filterArguments.size() == 1,
|
||||
"Expected 1 but got: %s", filterArguments.size());
|
||||
byte [] prefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0));
|
||||
return new PrefixFilter(prefix);
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
Bytes.writeByteArray(out, this.prefix);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,8 @@ package org.apache.hadoop.hbase.filter;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* This filter is used to filter based on the column qualifier. It takes an
|
||||
* operator (equal, greater, not equal, etc) and a byte [] comparator for the
|
||||
|
@ -65,4 +67,11 @@ public class QualifierFilter extends CompareFilter {
|
|||
}
|
||||
return ReturnCode.INCLUDE;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
|
||||
CompareOp compareOp = (CompareOp)arguments.get(0);
|
||||
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
|
||||
return new QualifierFilter(compareOp, comparator);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,11 @@ public class RowFilter extends CompareFilter {
|
|||
public boolean filterRow() {
|
||||
return this.filterOutRow;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
ArrayList arguments = CompareFilter.extractArguments(filterArguments);
|
||||
CompareOp compareOp = (CompareOp)arguments.get(0);
|
||||
WritableByteArrayComparable comparator = (WritableByteArrayComparable)arguments.get(1);
|
||||
return new RowFilter(compareOp, comparator);
|
||||
}
|
||||
}
|
|
@ -23,6 +23,8 @@ package org.apache.hadoop.hbase.filter;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
||||
/**
|
||||
* A {@link Filter} that checks a single column value, but does not emit the
|
||||
* tested column. This will enable a performance boost over
|
||||
|
@ -85,4 +87,18 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter {
|
|||
}
|
||||
return superRetCode;
|
||||
}
|
||||
|
||||
public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
|
||||
SingleColumnValueFilter tempFilter = (SingleColumnValueFilter)
|
||||
SingleColumnValueFilter.createFilterFromArguments(filterArguments);
|
||||
SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter (
|
||||
tempFilter.getFamily(), tempFilter.getQualifier(),
|
||||
tempFilter.getOperator(), tempFilter.getComparator());
|
||||
|
||||
if (filterArguments.size() == 6) {
|
||||
filter.setFilterIfMissing(tempFilter.getFilterIfMissing());
|
||||
filter.setLatestVersionOnly(tempFilter.getLatestVersionOnly());
|
||||
}
|
||||
return filter;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,6 +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}
|
||||
|
@ -247,6 +250,36 @@ public class SingleColumnValueFilter extends FilterBase {
|
|||
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) {
|
||||
|
|
|
@ -6,8 +6,10 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.TreeSet;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* Filter that returns only cells whose timestamp (version) is
|
||||
|
@ -41,6 +43,9 @@ public class TimestampsFilter extends FilterBase {
|
|||
* @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();
|
||||
}
|
||||
|
@ -80,6 +85,15 @@ public class TimestampsFilter extends FilterBase {
|
|||
return ReturnCode.SKIP;
|
||||
}
|
||||
|
||||
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();
|
||||
|
|
|
@ -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 CompareFilter {
|
|||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.PrefixFilter;
|
||||
import org.apache.hadoop.hbase.filter.WhileMatchFilter;
|
||||
import org.apache.hadoop.hbase.filter.ParseFilter;
|
||||
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
|
||||
import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
|
||||
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
|
||||
|
@ -748,6 +749,10 @@ public class ThriftServer {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (tScan.isSetFilterString()) {
|
||||
ParseFilter parseFilter = new ParseFilter();
|
||||
scan.setFilter(parseFilter.parseFilterString(tScan.getFilterString()));
|
||||
}
|
||||
return addScanner(table.getScanner(scan));
|
||||
} catch (IOException e) {
|
||||
throw new IOError(e.getMessage());
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -20,31 +20,40 @@ import java.util.Arrays;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.thrift.*;
|
||||
import org.apache.thrift.async.*;
|
||||
import org.apache.thrift.meta_data.*;
|
||||
import org.apache.thrift.transport.*;
|
||||
import org.apache.thrift.protocol.*;
|
||||
|
||||
/**
|
||||
* A Scan object is used to specify scanner parameters when opening a scanner.
|
||||
*/
|
||||
public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable {
|
||||
private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TScan");
|
||||
public class TScan implements TBase<TScan, TScan._Fields>, java.io.Serializable, Cloneable {
|
||||
private static final TStruct STRUCT_DESC = new TStruct("TScan");
|
||||
|
||||
private static final org.apache.thrift.protocol.TField START_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("startRow", org.apache.thrift.protocol.TType.STRING, (short)1);
|
||||
private static final org.apache.thrift.protocol.TField STOP_ROW_FIELD_DESC = new org.apache.thrift.protocol.TField("stopRow", org.apache.thrift.protocol.TType.STRING, (short)2);
|
||||
private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)3);
|
||||
private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)4);
|
||||
private static final org.apache.thrift.protocol.TField CACHING_FIELD_DESC = new org.apache.thrift.protocol.TField("caching", org.apache.thrift.protocol.TType.I32, (short)5);
|
||||
private static final TField START_ROW_FIELD_DESC = new TField("startRow", TType.STRING, (short)1);
|
||||
private static final TField STOP_ROW_FIELD_DESC = new TField("stopRow", TType.STRING, (short)2);
|
||||
private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)3);
|
||||
private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.LIST, (short)4);
|
||||
private static final TField CACHING_FIELD_DESC = new TField("caching", TType.I32, (short)5);
|
||||
private static final TField FILTER_STRING_FIELD_DESC = new TField("filterString", TType.STRING, (short)6);
|
||||
|
||||
public ByteBuffer startRow;
|
||||
public ByteBuffer stopRow;
|
||||
public long timestamp;
|
||||
public List<ByteBuffer> columns;
|
||||
public int caching;
|
||||
public ByteBuffer filterString;
|
||||
|
||||
/** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
|
||||
public enum _Fields implements org.apache.thrift.TFieldIdEnum {
|
||||
public enum _Fields implements TFieldIdEnum {
|
||||
START_ROW((short)1, "startRow"),
|
||||
STOP_ROW((short)2, "stopRow"),
|
||||
TIMESTAMP((short)3, "timestamp"),
|
||||
COLUMNS((short)4, "columns"),
|
||||
CACHING((short)5, "caching");
|
||||
CACHING((short)5, "caching"),
|
||||
FILTER_STRING((short)6, "filterString");
|
||||
|
||||
private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
|
||||
|
||||
|
@ -69,6 +78,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return COLUMNS;
|
||||
case 5: // CACHING
|
||||
return CACHING;
|
||||
case 6: // FILTER_STRING
|
||||
return FILTER_STRING;
|
||||
default:
|
||||
return null;
|
||||
}
|
||||
|
@ -113,22 +124,24 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
private static final int __CACHING_ISSET_ID = 1;
|
||||
private BitSet __isset_bit_vector = new BitSet(2);
|
||||
|
||||
public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
|
||||
public static final Map<_Fields, FieldMetaData> metaDataMap;
|
||||
static {
|
||||
Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.START_ROW, new org.apache.thrift.meta_data.FieldMetaData("startRow", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
|
||||
tmpMap.put(_Fields.STOP_ROW, new org.apache.thrift.meta_data.FieldMetaData("stopRow", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text")));
|
||||
tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
|
||||
tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "Text"))));
|
||||
tmpMap.put(_Fields.CACHING, new org.apache.thrift.meta_data.FieldMetaData("caching", org.apache.thrift.TFieldRequirementType.OPTIONAL,
|
||||
new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
|
||||
Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
|
||||
tmpMap.put(_Fields.START_ROW, new FieldMetaData("startRow", TFieldRequirementType.OPTIONAL,
|
||||
new FieldValueMetaData(TType.STRING , "Text")));
|
||||
tmpMap.put(_Fields.STOP_ROW, new FieldMetaData("stopRow", TFieldRequirementType.OPTIONAL,
|
||||
new FieldValueMetaData(TType.STRING , "Text")));
|
||||
tmpMap.put(_Fields.TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.OPTIONAL,
|
||||
new FieldValueMetaData(TType.I64)));
|
||||
tmpMap.put(_Fields.COLUMNS, new FieldMetaData("columns", TFieldRequirementType.OPTIONAL,
|
||||
new ListMetaData(TType.LIST,
|
||||
new FieldValueMetaData(TType.STRING , "Text"))));
|
||||
tmpMap.put(_Fields.CACHING, new FieldMetaData("caching", TFieldRequirementType.OPTIONAL,
|
||||
new FieldValueMetaData(TType.I32)));
|
||||
tmpMap.put(_Fields.FILTER_STRING, new FieldMetaData("filterString", TFieldRequirementType.OPTIONAL,
|
||||
new FieldValueMetaData(TType.STRING , "Text")));
|
||||
metaDataMap = Collections.unmodifiableMap(tmpMap);
|
||||
org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TScan.class, metaDataMap);
|
||||
FieldMetaData.addStructMetaDataMap(TScan.class, metaDataMap);
|
||||
}
|
||||
|
||||
public TScan() {
|
||||
|
@ -155,6 +168,9 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.columns = __this__columns;
|
||||
}
|
||||
this.caching = other.caching;
|
||||
if (other.isSetFilterString()) {
|
||||
this.filterString = other.filterString;
|
||||
}
|
||||
}
|
||||
|
||||
public TScan deepCopy() {
|
||||
|
@ -170,19 +186,20 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.columns = null;
|
||||
setCachingIsSet(false);
|
||||
this.caching = 0;
|
||||
this.filterString = null;
|
||||
}
|
||||
|
||||
public byte[] getStartRow() {
|
||||
setStartRow(org.apache.thrift.TBaseHelper.rightSize(startRow));
|
||||
return startRow == null ? null : startRow.array();
|
||||
setStartRow(TBaseHelper.rightSize(startRow));
|
||||
return startRow.array();
|
||||
}
|
||||
|
||||
public ByteBuffer bufferForStartRow() {
|
||||
public ByteBuffer BufferForStartRow() {
|
||||
return startRow;
|
||||
}
|
||||
|
||||
public TScan setStartRow(byte[] startRow) {
|
||||
setStartRow(startRow == null ? (ByteBuffer)null : ByteBuffer.wrap(startRow));
|
||||
setStartRow(ByteBuffer.wrap(startRow));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -195,7 +212,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.startRow = null;
|
||||
}
|
||||
|
||||
/** Returns true if field startRow is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field startRow is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetStartRow() {
|
||||
return this.startRow != null;
|
||||
}
|
||||
|
@ -207,16 +224,16 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
}
|
||||
|
||||
public byte[] getStopRow() {
|
||||
setStopRow(org.apache.thrift.TBaseHelper.rightSize(stopRow));
|
||||
return stopRow == null ? null : stopRow.array();
|
||||
setStopRow(TBaseHelper.rightSize(stopRow));
|
||||
return stopRow.array();
|
||||
}
|
||||
|
||||
public ByteBuffer bufferForStopRow() {
|
||||
public ByteBuffer BufferForStopRow() {
|
||||
return stopRow;
|
||||
}
|
||||
|
||||
public TScan setStopRow(byte[] stopRow) {
|
||||
setStopRow(stopRow == null ? (ByteBuffer)null : ByteBuffer.wrap(stopRow));
|
||||
setStopRow(ByteBuffer.wrap(stopRow));
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -229,7 +246,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.stopRow = null;
|
||||
}
|
||||
|
||||
/** Returns true if field stopRow is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field stopRow is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetStopRow() {
|
||||
return this.stopRow != null;
|
||||
}
|
||||
|
@ -254,7 +271,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
__isset_bit_vector.clear(__TIMESTAMP_ISSET_ID);
|
||||
}
|
||||
|
||||
/** Returns true if field timestamp is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field timestamp is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetTimestamp() {
|
||||
return __isset_bit_vector.get(__TIMESTAMP_ISSET_ID);
|
||||
}
|
||||
|
@ -291,7 +308,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
this.columns = null;
|
||||
}
|
||||
|
||||
/** Returns true if field columns is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field columns is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetColumns() {
|
||||
return this.columns != null;
|
||||
}
|
||||
|
@ -316,7 +333,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
__isset_bit_vector.clear(__CACHING_ISSET_ID);
|
||||
}
|
||||
|
||||
/** Returns true if field caching is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field caching is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetCaching() {
|
||||
return __isset_bit_vector.get(__CACHING_ISSET_ID);
|
||||
}
|
||||
|
@ -325,6 +342,40 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
__isset_bit_vector.set(__CACHING_ISSET_ID, value);
|
||||
}
|
||||
|
||||
public byte[] getFilterString() {
|
||||
setFilterString(TBaseHelper.rightSize(filterString));
|
||||
return filterString.array();
|
||||
}
|
||||
|
||||
public ByteBuffer BufferForFilterString() {
|
||||
return filterString;
|
||||
}
|
||||
|
||||
public TScan setFilterString(byte[] filterString) {
|
||||
setFilterString(ByteBuffer.wrap(filterString));
|
||||
return this;
|
||||
}
|
||||
|
||||
public TScan setFilterString(ByteBuffer filterString) {
|
||||
this.filterString = filterString;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void unsetFilterString() {
|
||||
this.filterString = null;
|
||||
}
|
||||
|
||||
/** Returns true if field filterString is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSetFilterString() {
|
||||
return this.filterString != null;
|
||||
}
|
||||
|
||||
public void setFilterStringIsSet(boolean value) {
|
||||
if (!value) {
|
||||
this.filterString = null;
|
||||
}
|
||||
}
|
||||
|
||||
public void setFieldValue(_Fields field, Object value) {
|
||||
switch (field) {
|
||||
case START_ROW:
|
||||
|
@ -367,6 +418,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
}
|
||||
break;
|
||||
|
||||
case FILTER_STRING:
|
||||
if (value == null) {
|
||||
unsetFilterString();
|
||||
} else {
|
||||
setFilterString((ByteBuffer)value);
|
||||
}
|
||||
break;
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -387,11 +446,14 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
case CACHING:
|
||||
return new Integer(getCaching());
|
||||
|
||||
case FILTER_STRING:
|
||||
return getFilterString();
|
||||
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
||||
/** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
|
||||
/** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
|
||||
public boolean isSet(_Fields field) {
|
||||
if (field == null) {
|
||||
throw new IllegalArgumentException();
|
||||
|
@ -408,6 +470,8 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return isSetColumns();
|
||||
case CACHING:
|
||||
return isSetCaching();
|
||||
case FILTER_STRING:
|
||||
return isSetFilterString();
|
||||
}
|
||||
throw new IllegalStateException();
|
||||
}
|
||||
|
@ -470,6 +534,15 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return false;
|
||||
}
|
||||
|
||||
boolean this_present_filterString = true && this.isSetFilterString();
|
||||
boolean that_present_filterString = true && that.isSetFilterString();
|
||||
if (this_present_filterString || that_present_filterString) {
|
||||
if (!(this_present_filterString && that_present_filterString))
|
||||
return false;
|
||||
if (!this.filterString.equals(that.filterString))
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -491,7 +564,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
if (isSetStartRow()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRow, typedOther.startRow);
|
||||
lastComparison = TBaseHelper.compareTo(this.startRow, typedOther.startRow);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -501,7 +574,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
if (isSetStopRow()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stopRow, typedOther.stopRow);
|
||||
lastComparison = TBaseHelper.compareTo(this.stopRow, typedOther.stopRow);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -511,7 +584,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
if (isSetTimestamp()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp);
|
||||
lastComparison = TBaseHelper.compareTo(this.timestamp, typedOther.timestamp);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -521,7 +594,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
if (isSetColumns()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, typedOther.columns);
|
||||
lastComparison = TBaseHelper.compareTo(this.columns, typedOther.columns);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -531,7 +604,17 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return lastComparison;
|
||||
}
|
||||
if (isSetCaching()) {
|
||||
lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.caching, typedOther.caching);
|
||||
lastComparison = TBaseHelper.compareTo(this.caching, typedOther.caching);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
}
|
||||
lastComparison = Boolean.valueOf(isSetFilterString()).compareTo(typedOther.isSetFilterString());
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
if (isSetFilterString()) {
|
||||
lastComparison = TBaseHelper.compareTo(this.filterString, typedOther.filterString);
|
||||
if (lastComparison != 0) {
|
||||
return lastComparison;
|
||||
}
|
||||
|
@ -543,42 +626,42 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
return _Fields.findByThriftId(fieldId);
|
||||
}
|
||||
|
||||
public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
|
||||
org.apache.thrift.protocol.TField field;
|
||||
public void read(TProtocol iprot) throws TException {
|
||||
TField field;
|
||||
iprot.readStructBegin();
|
||||
while (true)
|
||||
{
|
||||
field = iprot.readFieldBegin();
|
||||
if (field.type == org.apache.thrift.protocol.TType.STOP) {
|
||||
if (field.type == TType.STOP) {
|
||||
break;
|
||||
}
|
||||
switch (field.id) {
|
||||
case 1: // START_ROW
|
||||
if (field.type == org.apache.thrift.protocol.TType.STRING) {
|
||||
if (field.type == TType.STRING) {
|
||||
this.startRow = iprot.readBinary();
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 2: // STOP_ROW
|
||||
if (field.type == org.apache.thrift.protocol.TType.STRING) {
|
||||
if (field.type == TType.STRING) {
|
||||
this.stopRow = iprot.readBinary();
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 3: // TIMESTAMP
|
||||
if (field.type == org.apache.thrift.protocol.TType.I64) {
|
||||
if (field.type == TType.I64) {
|
||||
this.timestamp = iprot.readI64();
|
||||
setTimestampIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 4: // COLUMNS
|
||||
if (field.type == org.apache.thrift.protocol.TType.LIST) {
|
||||
if (field.type == TType.LIST) {
|
||||
{
|
||||
org.apache.thrift.protocol.TList _list9 = iprot.readListBegin();
|
||||
TList _list9 = iprot.readListBegin();
|
||||
this.columns = new ArrayList<ByteBuffer>(_list9.size);
|
||||
for (int _i10 = 0; _i10 < _list9.size; ++_i10)
|
||||
{
|
||||
|
@ -589,19 +672,26 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
iprot.readListEnd();
|
||||
}
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 5: // CACHING
|
||||
if (field.type == org.apache.thrift.protocol.TType.I32) {
|
||||
if (field.type == TType.I32) {
|
||||
this.caching = iprot.readI32();
|
||||
setCachingIsSet(true);
|
||||
} else {
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
case 6: // FILTER_STRING
|
||||
if (field.type == TType.STRING) {
|
||||
this.filterString = iprot.readBinary();
|
||||
} else {
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type);
|
||||
TProtocolUtil.skip(iprot, field.type);
|
||||
}
|
||||
iprot.readFieldEnd();
|
||||
}
|
||||
|
@ -611,7 +701,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
validate();
|
||||
}
|
||||
|
||||
public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
|
||||
public void write(TProtocol oprot) throws TException {
|
||||
validate();
|
||||
|
||||
oprot.writeStructBegin(STRUCT_DESC);
|
||||
|
@ -638,7 +728,7 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
if (isSetColumns()) {
|
||||
oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
|
||||
{
|
||||
oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, this.columns.size()));
|
||||
oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
|
||||
for (ByteBuffer _iter12 : this.columns)
|
||||
{
|
||||
oprot.writeBinary(_iter12);
|
||||
|
@ -653,6 +743,13 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
oprot.writeI32(this.caching);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
if (this.filterString != null) {
|
||||
if (isSetFilterString()) {
|
||||
oprot.writeFieldBegin(FILTER_STRING_FIELD_DESC);
|
||||
oprot.writeBinary(this.filterString);
|
||||
oprot.writeFieldEnd();
|
||||
}
|
||||
}
|
||||
oprot.writeFieldStop();
|
||||
oprot.writeStructEnd();
|
||||
}
|
||||
|
@ -703,31 +800,23 @@ public class TScan implements org.apache.thrift.TBase<TScan, TScan._Fields>, jav
|
|||
sb.append(this.caching);
|
||||
first = false;
|
||||
}
|
||||
if (isSetFilterString()) {
|
||||
if (!first) sb.append(", ");
|
||||
sb.append("filterString:");
|
||||
if (this.filterString == null) {
|
||||
sb.append("null");
|
||||
} else {
|
||||
sb.append(this.filterString);
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
sb.append(")");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public void validate() throws org.apache.thrift.TException {
|
||||
public void validate() throws TException {
|
||||
// check for required fields
|
||||
}
|
||||
|
||||
private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
|
||||
try {
|
||||
write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
|
||||
} catch (org.apache.thrift.TException te) {
|
||||
throw new java.io.IOException(te);
|
||||
}
|
||||
}
|
||||
|
||||
private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
|
||||
try {
|
||||
// it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
|
||||
__isset_bit_vector = new BitSet(1);
|
||||
read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
|
||||
} catch (org.apache.thrift.TException te) {
|
||||
throw new java.io.IOException(te);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -124,7 +124,8 @@ struct TScan {
|
|||
2:optional Text stopRow,
|
||||
3:optional i64 timestamp,
|
||||
4:optional list<Text> columns,
|
||||
5:optional i32 caching
|
||||
5:optional i32 caching,
|
||||
6:optional Text filterString
|
||||
}
|
||||
|
||||
//
|
||||
|
|
|
@ -236,7 +236,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(org.apache.hadoop.hbase.filter.ParseFilter.new.parseFilterString(filter))
|
||||
end
|
||||
|
||||
scan.setTimeStamp(timestamp) if timestamp
|
||||
scan.setCacheBlocks(cache)
|
||||
scan.setMaxVersions(versions) if versions > 1
|
||||
|
|
|
@ -26,17 +26,25 @@ module Shell
|
|||
Scan a table; pass table name and optionally a dictionary of scanner
|
||||
specifications. Scanner specifications may include one or more of:
|
||||
TIMERANGE, FILTER, LIMIT, STARTROW, STOPROW, TIMESTAMP, MAXLENGTH,
|
||||
or COLUMNS. If no columns are specified, all columns will be scanned.
|
||||
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 in the
|
||||
Filter Language document attached to the HBASE-4176 JIRA
|
||||
2. Using the entire package name of the filter.
|
||||
|
||||
Some examples:
|
||||
|
||||
hbase> scan '.META.'
|
||||
hbase> scan '.META.', {COLUMNS => 'info:regioninfo'}
|
||||
hbase> scan 't1', {COLUMNS => ['c1', 'c2'], LIMIT => 10, STARTROW => 'xyz'}
|
||||
hbase> scan 't1', {FILTER => org.apache.hadoop.hbase.filter.ColumnPaginationFilter.new(1, 0)}
|
||||
hbase> scan 't1', {COLUMNS => 'c1', TIMERANGE => [1303668804, 1303668904]}
|
||||
hbase> scan 't1', {FILTER => "(PrefixFilter ('row2') AND (QualifierFilter (>=, 'binary:xyz'))) AND (TimestampsFilter ( 123, 456))"}
|
||||
hbase> scan 't1', {FILTER => org.apache.hadoop.hbase.filter.ColumnPaginationFilter.new(1, 0)}
|
||||
|
||||
For experts, there is an additional option -- CACHE_BLOCKS -- which
|
||||
switches block caching for the scanner on (true) or off (false). By
|
||||
|
|
|
@ -0,0 +1,663 @@
|
|||
/**
|
||||
* 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);
|
||||
List<Long> timestamps = timestampsFilter.getTimestamps();
|
||||
assertEquals(timestamps.size(), 2);
|
||||
assertEquals(timestamps.get(0), 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 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);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue