You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2013/02/25 23:50:29 UTC
svn commit: r1449950 [18/35] - in /hbase/trunk: ./ hbase-client/
hbase-client/src/ hbase-client/src/main/ hbase-client/src/main/java/
hbase-client/src/main/java/org/ hbase-client/src/main/java/org/apache/
hbase-client/src/main/java/org/apache/hadoop/ h...
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,198 @@
+/*
+ * 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 com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.TreeSet;
+
+/**
+ * This filter is used for selecting only those keys with columns that matches
+ * a particular prefix. For example, if prefix is 'an', it will pass keys will
+ * columns like 'and', 'anti' but not keys with columns like 'ball', 'act'.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class MultipleColumnPrefixFilter extends FilterBase {
+ protected byte [] hint = null;
+ protected TreeSet<byte []> sortedPrefixes = createTreeSet();
+ private final static int MAX_LOG_PREFIXES = 5;
+
+ public MultipleColumnPrefixFilter(final byte [][] prefixes) {
+ if (prefixes != null) {
+ for (int i = 0; i < prefixes.length; i++) {
+ if (!sortedPrefixes.add(prefixes[i]))
+ throw new IllegalArgumentException ("prefixes must be distinct");
+ }
+ }
+ }
+
+ public byte [][] getPrefix() {
+ int count = 0;
+ byte [][] temp = new byte [sortedPrefixes.size()][];
+ for (byte [] prefixes : sortedPrefixes) {
+ temp [count++] = prefixes;
+ }
+ return temp;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue kv) {
+ if (sortedPrefixes.size() == 0 || kv.getBuffer() == null) {
+ return ReturnCode.INCLUDE;
+ } else {
+ return filterColumn(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
+ }
+ }
+
+ public ReturnCode filterColumn(byte[] buffer, int qualifierOffset, int qualifierLength) {
+ byte [] qualifier = Arrays.copyOfRange(buffer, qualifierOffset,
+ qualifierLength + qualifierOffset);
+ TreeSet<byte []> lesserOrEqualPrefixes =
+ (TreeSet<byte []>) sortedPrefixes.headSet(qualifier, true);
+
+ if (lesserOrEqualPrefixes.size() != 0) {
+ byte [] largestPrefixSmallerThanQualifier = lesserOrEqualPrefixes.last();
+
+ if (Bytes.startsWith(qualifier, largestPrefixSmallerThanQualifier)) {
+ return ReturnCode.INCLUDE;
+ }
+
+ if (lesserOrEqualPrefixes.size() == sortedPrefixes.size()) {
+ return ReturnCode.NEXT_ROW;
+ } else {
+ hint = sortedPrefixes.higher(largestPrefixSmallerThanQualifier);
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+ } else {
+ hint = sortedPrefixes.first();
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ byte [][] prefixes = new byte [filterArguments.size()][];
+ for (int i = 0 ; i < filterArguments.size(); i++) {
+ byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(i));
+ prefixes[i] = columnPrefix;
+ }
+ return new MultipleColumnPrefixFilter(prefixes);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.MultipleColumnPrefixFilter.Builder builder =
+ FilterProtos.MultipleColumnPrefixFilter.newBuilder();
+ for (byte [] element : sortedPrefixes) {
+ if (element != null) builder.addSortedPrefixes(ByteString.copyFrom(element));
+ }
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link MultipleColumnPrefixFilter} instance
+ * @return An instance of {@link MultipleColumnPrefixFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static MultipleColumnPrefixFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.MultipleColumnPrefixFilter proto;
+ try {
+ proto = FilterProtos.MultipleColumnPrefixFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ int numPrefixes = proto.getSortedPrefixesCount();
+ byte [][] prefixes = new byte[numPrefixes][];
+ for (int i = 0; i < numPrefixes; ++i) {
+ prefixes[i] = proto.getSortedPrefixes(i).toByteArray();
+ }
+
+ return new MultipleColumnPrefixFilter(prefixes);
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof MultipleColumnPrefixFilter)) return false;
+
+ MultipleColumnPrefixFilter other = (MultipleColumnPrefixFilter)o;
+ return this.sortedPrefixes.equals(other.sortedPrefixes);
+ }
+
+ public KeyValue getNextKeyHint(KeyValue kv) {
+ return KeyValue.createFirstOnRow(
+ kv.getBuffer(), kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
+ kv.getFamilyOffset(), kv.getFamilyLength(), hint, 0, hint.length);
+ }
+
+ public TreeSet<byte []> createTreeSet() {
+ return new TreeSet<byte []>(new Comparator<Object>() {
+ @Override
+ public int compare (Object o1, Object o2) {
+ if (o1 == null || o2 == null)
+ throw new IllegalArgumentException ("prefixes can't be null");
+
+ byte [] b1 = (byte []) o1;
+ byte [] b2 = (byte []) o2;
+ return Bytes.compareTo (b1, 0, b1.length, b2, 0, b2.length);
+ }
+ });
+ }
+
+ @Override
+ public String toString() {
+ return toString(MAX_LOG_PREFIXES);
+ }
+
+ protected String toString(int maxPrefixes) {
+ StringBuilder prefixes = new StringBuilder();
+
+ int count = 0;
+ for (byte[] ba : this.sortedPrefixes) {
+ if (count >= maxPrefixes) {
+ break;
+ }
+ ++count;
+ prefixes.append(Bytes.toStringBinary(ba));
+ if (count < this.sortedPrefixes.size() && count < maxPrefixes) {
+ prefixes.append(", ");
+ }
+ }
+
+ return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(),
+ count, this.sortedPrefixes.size(), prefixes.toString());
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/NullComparator.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,92 @@
+/*
+ *
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+
+/**
+ * A binary comparator which lexicographically compares against the specified
+ * byte array using {@link org.apache.hadoop.hbase.util.Bytes#compareTo(byte[], byte[])}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class NullComparator extends ByteArrayComparable {
+
+ public NullComparator() {
+ super(new byte[0]);
+ }
+
+ @Override
+ public int compareTo(byte[] value) {
+ return value != null ? 1 : 0;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ return obj == null;
+ }
+
+ @Override
+ public int compareTo(byte[] value, int offset, int length) {
+ throw new UnsupportedOperationException();
+ }
+
+ /**
+ * @return The comparator serialized using pb
+ */
+ public byte [] toByteArray() {
+ ComparatorProtos.NullComparator.Builder builder =
+ ComparatorProtos.NullComparator.newBuilder();
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link NullComparator} instance
+ * @return An instance of {@link NullComparator} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static NullComparator parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ ComparatorProtos.NullComparator proto;
+ try {
+ proto = ComparatorProtos.NullComparator.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new NullComparator();
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the comparator that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(ByteArrayComparable other) {
+ if (other == this) return true;
+ if (!(other instanceof NullComparator)) return false;
+
+ return super.areSerializedFieldsEqual(other);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,124 @@
+/**
+ *
+ * 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 com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.util.ArrayList;
+/**
+ * Implementation of Filter interface that limits results to a specific page
+ * size. It terminates scanning once the number of filter-passed rows is >
+ * the given page size.
+ * <p>
+ * Note that this filter cannot guarantee that the number of results returned
+ * to a client are <= page size. This is because the filter is applied
+ * separately on different region servers. It does however optimize the scan of
+ * individual HRegions by making sure that the page size is never exceeded
+ * locally.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PageFilter extends FilterBase {
+ private long pageSize = Long.MAX_VALUE;
+ private int rowsAccepted = 0;
+
+ /**
+ * Constructor that takes a maximum page size.
+ *
+ * @param pageSize Maximum result size.
+ */
+ public PageFilter(final long pageSize) {
+ Preconditions.checkArgument(pageSize >= 0, "must be positive %s", pageSize);
+ this.pageSize = pageSize;
+ }
+
+ public long getPageSize() {
+ return pageSize;
+ }
+
+ public boolean filterAllRemaining() {
+ return this.rowsAccepted >= this.pageSize;
+ }
+
+ public boolean filterRow() {
+ this.rowsAccepted++;
+ return this.rowsAccepted > this.pageSize;
+ }
+
+ public boolean hasFilterRow() {
+ return true;
+ }
+
+ 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);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.PageFilter.Builder builder =
+ FilterProtos.PageFilter.newBuilder();
+ builder.setPageSize(this.pageSize);
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link PageFilter} instance
+ * @return An instance of {@link PageFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static PageFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.PageFilter proto;
+ try {
+ proto = FilterProtos.PageFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new PageFilter(proto.getPageSize());
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof PageFilter)) return false;
+
+ PageFilter other = (PageFilter)o;
+ return this.getPageSize() == other.getPageSize();
+ }
+
+ @Override
+ public String toString() {
+ return this.getClass().getSimpleName() + " " + this.pageSize;
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseConstants.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,263 @@
+/**
+ *
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.nio.ByteBuffer;
+
+/**
+ * ParseConstants holds a bunch of constants related to parsing Filter Strings
+ * Used by {@link ParseFilter}
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public final class ParseConstants {
+
+ /**
+ * ASCII code for LPAREN
+ */
+ public static final int LPAREN = '(';
+
+ /**
+ * ASCII code for RPAREN
+ */
+ public static final int RPAREN = ')';
+
+ /**
+ * ASCII code for whitespace
+ */
+ public static final int WHITESPACE = ' ';
+
+ /**
+ * ASCII code for tab
+ */
+ public static final int TAB = '\t';
+
+ /**
+ * ASCII code for 'A'
+ */
+ public static final int A = 'A';
+
+ /**
+ * ASCII code for 'N'
+ */
+ public static final int N = 'N';
+
+ /**
+ * ASCII code for 'D'
+ */
+ public static final int D = 'D';
+
+ /**
+ * ASCII code for 'O'
+ */
+ public static final int O = 'O';
+
+ /**
+ * ASCII code for 'R'
+ */
+ public static final int R = 'R';
+
+ /**
+ * ASCII code for 'S'
+ */
+ public static final int S = 'S';
+
+ /**
+ * ASCII code for 'K'
+ */
+ public static final int K = 'K';
+
+ /**
+ * ASCII code for 'I'
+ */
+ public static final int I = 'I';
+
+ /**
+ * ASCII code for 'P'
+ */
+ public static final int P = 'P';
+
+ /**
+ * SKIP Array
+ */
+ public static final byte [] SKIP_ARRAY = new byte [ ] {'S', 'K', 'I', 'P'};
+ public static final ByteBuffer SKIP_BUFFER = ByteBuffer.wrap(SKIP_ARRAY);
+
+ /**
+ * ASCII code for 'W'
+ */
+ public static final int W = 'W';
+
+ /**
+ * ASCII code for 'H'
+ */
+ public static final int H = 'H';
+
+ /**
+ * ASCII code for 'L'
+ */
+ public static final int L = 'L';
+
+ /**
+ * ASCII code for 'E'
+ */
+ public static final int E = 'E';
+
+ /**
+ * WHILE Array
+ */
+ public static final byte [] WHILE_ARRAY = new byte [] {'W', 'H', 'I', 'L', 'E'};
+ public static final ByteBuffer WHILE_BUFFER = ByteBuffer.wrap(WHILE_ARRAY);
+
+ /**
+ * OR Array
+ */
+ public static final byte [] OR_ARRAY = new byte [] {'O','R'};
+ public static final ByteBuffer OR_BUFFER = ByteBuffer.wrap(OR_ARRAY);
+
+ /**
+ * AND Array
+ */
+ public static final byte [] AND_ARRAY = new byte [] {'A','N', 'D'};
+ public static final ByteBuffer AND_BUFFER = ByteBuffer.wrap(AND_ARRAY);
+
+ /**
+ * ASCII code for Backslash
+ */
+ public static final int BACKSLASH = '\\';
+
+ /**
+ * ASCII code for a single quote
+ */
+ public static final int SINGLE_QUOTE = '\'';
+
+ /**
+ * ASCII code for a comma
+ */
+ public static final int COMMA = ',';
+
+ /**
+ * LESS_THAN Array
+ */
+ public static final byte [] LESS_THAN_ARRAY = new byte [] {'<'};
+ public static final ByteBuffer LESS_THAN_BUFFER = ByteBuffer.wrap(LESS_THAN_ARRAY);
+
+ /**
+ * LESS_THAN_OR_EQUAL_TO Array
+ */
+ public static final byte [] LESS_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'<', '='};
+ public static final ByteBuffer LESS_THAN_OR_EQUAL_TO_BUFFER =
+ ByteBuffer.wrap(LESS_THAN_OR_EQUAL_TO_ARRAY);
+
+ /**
+ * GREATER_THAN Array
+ */
+ public static final byte [] GREATER_THAN_ARRAY = new byte [] {'>'};
+ public static final ByteBuffer GREATER_THAN_BUFFER = ByteBuffer.wrap(GREATER_THAN_ARRAY);
+
+ /**
+ * GREATER_THAN_OR_EQUAL_TO Array
+ */
+ public static final byte [] GREATER_THAN_OR_EQUAL_TO_ARRAY = new byte [] {'>', '='};
+ public static final ByteBuffer GREATER_THAN_OR_EQUAL_TO_BUFFER =
+ ByteBuffer.wrap(GREATER_THAN_OR_EQUAL_TO_ARRAY);
+
+ /**
+ * EQUAL_TO Array
+ */
+ public static final byte [] EQUAL_TO_ARRAY = new byte [] {'='};
+ public static final ByteBuffer EQUAL_TO_BUFFER = ByteBuffer.wrap(EQUAL_TO_ARRAY);
+
+ /**
+ * NOT_EQUAL_TO Array
+ */
+ public static final byte [] NOT_EQUAL_TO_ARRAY = new byte [] {'!', '='};
+ public static final ByteBuffer NOT_EQUAL_TO_BUFFER = ByteBuffer.wrap(NOT_EQUAL_TO_ARRAY);
+
+ /**
+ * ASCII code for equal to (=)
+ */
+ public static final int EQUAL_TO = '=';
+
+ /**
+ * AND Byte Array
+ */
+ public static final byte [] AND = new byte [] {'A','N','D'};
+
+ /**
+ * OR Byte Array
+ */
+ public static final byte [] OR = new byte [] {'O', 'R'};
+
+ /**
+ * LPAREN Array
+ */
+ public static final byte [] LPAREN_ARRAY = new byte [] {'('};
+ public static final ByteBuffer LPAREN_BUFFER = ByteBuffer.wrap(LPAREN_ARRAY);
+
+ /**
+ * ASCII code for colon (:)
+ */
+ public static final int COLON = ':';
+
+ /**
+ * ASCII code for Zero
+ */
+ public static final int ZERO = '0';
+
+ /**
+ * ASCII code foe Nine
+ */
+ public static final int NINE = '9';
+
+ /**
+ * BinaryType byte array
+ */
+ public static final byte [] binaryType = new byte [] {'b','i','n','a','r','y'};
+
+ /**
+ * BinaryPrefixType byte array
+ */
+ public static final byte [] binaryPrefixType = new byte [] {'b','i','n','a','r','y',
+ 'p','r','e','f','i','x'};
+
+ /**
+ * RegexStringType byte array
+ */
+ public static final byte [] regexStringType = new byte [] {'r','e','g','e', 'x',
+ 's','t','r','i','n','g'};
+
+ /**
+ * SubstringType byte array
+ */
+ public static final byte [] substringType = new byte [] {'s','u','b','s','t','r','i','n','g'};
+
+ /**
+ * ASCII for Minus Sign
+ */
+ public static final int MINUS_SIGN = '-';
+
+ /**
+ * Package containing filters
+ */
+ public static final String FILTER_PACKAGE = "org.apache.hadoop.hbase.filter";
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ParseFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,865 @@
+/**
+ *
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EmptyStackException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+
+/**
+ * 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
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class ParseFilter {
+ private static final Log LOG = LogFactory.getLog(ParseFilter.class);
+
+ private static HashMap<ByteBuffer, Integer> operatorPrecedenceHashMap;
+ private static HashMap<String, String> filterHashMap;
+
+ static {
+ // Registers all the filter supported by the Filter Language
+ filterHashMap = new HashMap<String, String>();
+ filterHashMap.put("KeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "KeyOnlyFilter");
+ filterHashMap.put("FirstKeyOnlyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "FirstKeyOnlyFilter");
+ filterHashMap.put("PrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "PrefixFilter");
+ filterHashMap.put("ColumnPrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnPrefixFilter");
+ filterHashMap.put("MultipleColumnPrefixFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "MultipleColumnPrefixFilter");
+ filterHashMap.put("ColumnCountGetFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnCountGetFilter");
+ filterHashMap.put("PageFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "PageFilter");
+ filterHashMap.put("ColumnPaginationFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnPaginationFilter");
+ filterHashMap.put("InclusiveStopFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "InclusiveStopFilter");
+ filterHashMap.put("TimestampsFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "TimestampsFilter");
+ filterHashMap.put("RowFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "RowFilter");
+ filterHashMap.put("FamilyFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "FamilyFilter");
+ filterHashMap.put("QualifierFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "QualifierFilter");
+ filterHashMap.put("ValueFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ValueFilter");
+ filterHashMap.put("ColumnRangeFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "ColumnRangeFilter");
+ filterHashMap.put("SingleColumnValueFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "SingleColumnValueFilter");
+ filterHashMap.put("SingleColumnValueExcludeFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "SingleColumnValueExcludeFilter");
+ filterHashMap.put("DependentColumnFilter", ParseConstants.FILTER_PACKAGE + "." +
+ "DependentColumnFilter");
+
+ // Creates the operatorPrecedenceHashMap
+ operatorPrecedenceHashMap = new HashMap<ByteBuffer, Integer>();
+ operatorPrecedenceHashMap.put(ParseConstants.SKIP_BUFFER, 1);
+ operatorPrecedenceHashMap.put(ParseConstants.WHILE_BUFFER, 1);
+ operatorPrecedenceHashMap.put(ParseConstants.AND_BUFFER, 2);
+ operatorPrecedenceHashMap.put(ParseConstants.OR_BUFFER, 3);
+ }
+
+ /**
+ * Parses the filterString and constructs a filter using it
+ * <p>
+ * @param filterString filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseFilterString (String filterString)
+ throws CharacterCodingException {
+ return parseFilterString(Bytes.toBytes(filterString));
+ }
+
+ /**
+ * Parses the filterString and constructs a filter using it
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseFilterString (byte [] filterStringAsByteArray)
+ throws CharacterCodingException {
+ // stack for the operators and parenthesis
+ Stack <ByteBuffer> operatorStack = new Stack<ByteBuffer>();
+ // stack for the filter objects
+ Stack <Filter> filterStack = new Stack<Filter>();
+
+ Filter filter = null;
+ for (int i=0; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.LPAREN) {
+ // LPAREN found
+ operatorStack.push(ParseConstants.LPAREN_BUFFER);
+ } else if (filterStringAsByteArray[i] == ParseConstants.WHITESPACE ||
+ filterStringAsByteArray[i] == ParseConstants.TAB) {
+ // WHITESPACE or TAB found
+ continue;
+ } else if (checkForOr(filterStringAsByteArray, i)) {
+ // OR found
+ i += ParseConstants.OR_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.OR_BUFFER);
+ operatorStack.push(ParseConstants.OR_BUFFER);
+ } else if (checkForAnd(filterStringAsByteArray, i)) {
+ // AND found
+ i += ParseConstants.AND_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.AND_BUFFER);
+ operatorStack.push(ParseConstants.AND_BUFFER);
+ } else if (checkForSkip(filterStringAsByteArray, i)) {
+ // SKIP found
+ i += ParseConstants.SKIP_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.SKIP_BUFFER);
+ operatorStack.push(ParseConstants.SKIP_BUFFER);
+ } else if (checkForWhile(filterStringAsByteArray, i)) {
+ // WHILE found
+ i += ParseConstants.WHILE_ARRAY.length - 1;
+ reduce(operatorStack, filterStack, ParseConstants.WHILE_BUFFER);
+ operatorStack.push(ParseConstants.WHILE_BUFFER);
+ } else if (filterStringAsByteArray[i] == ParseConstants.RPAREN) {
+ // RPAREN found
+ if (operatorStack.empty()) {
+ throw new IllegalArgumentException("Mismatched parenthesis");
+ }
+ ByteBuffer argumentOnTopOfStack = operatorStack.peek();
+ while (!(argumentOnTopOfStack.equals(ParseConstants.LPAREN_BUFFER))) {
+ filterStack.push(popArguments(operatorStack, filterStack));
+ if (operatorStack.empty()) {
+ throw new IllegalArgumentException("Mismatched parenthesis");
+ }
+ argumentOnTopOfStack = operatorStack.pop();
+ }
+ } else {
+ // SimpleFilterExpression found
+ byte [] filterSimpleExpression = extractFilterSimpleExpression(filterStringAsByteArray, i);
+ i+= (filterSimpleExpression.length - 1);
+ filter = parseSimpleFilterExpression(filterSimpleExpression);
+ filterStack.push(filter);
+ }
+ }
+
+ // Finished parsing filterString
+ while (!operatorStack.empty()) {
+ filterStack.push(popArguments(operatorStack, filterStack));
+ }
+ filter = filterStack.pop();
+ if (!filterStack.empty()) {
+ throw new IllegalArgumentException("Incorrect Filter String");
+ }
+ return filter;
+ }
+
+/**
+ * Extracts a simple filter expression from the filter string given by the user
+ * <p>
+ * A simpleFilterExpression is of the form: FilterName('arg', 'arg', 'arg')
+ * The user given filter string can have many simpleFilterExpressions combined
+ * using operators.
+ * <p>
+ * This function extracts a simpleFilterExpression from the
+ * larger filterString given the start offset of the simpler expression
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @param filterExpressionStartOffset start index of the simple filter expression
+ * @return byte array containing the simple filter expression
+ */
+ public byte [] extractFilterSimpleExpression (byte [] filterStringAsByteArray,
+ int filterExpressionStartOffset)
+ throws CharacterCodingException {
+ int quoteCount = 0;
+ for (int i=filterExpressionStartOffset; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.SINGLE_QUOTE) {
+ if (isQuoteUnescaped(filterStringAsByteArray, i)) {
+ quoteCount ++;
+ } else {
+ // To skip the next quote that has been escaped
+ i++;
+ }
+ }
+ if (filterStringAsByteArray[i] == ParseConstants.RPAREN && (quoteCount %2 ) == 0) {
+ byte [] filterSimpleExpression = new byte [i - filterExpressionStartOffset + 1];
+ Bytes.putBytes(filterSimpleExpression, 0, filterStringAsByteArray,
+ filterExpressionStartOffset, i-filterExpressionStartOffset + 1);
+ return filterSimpleExpression;
+ }
+ }
+ throw new IllegalArgumentException("Incorrect Filter String");
+ }
+
+/**
+ * Constructs a filter object given a simple filter expression
+ * <p>
+ * @param filterStringAsByteArray filter string given by the user
+ * @return filter object we constructed
+ */
+ public Filter parseSimpleFilterExpression (byte [] filterStringAsByteArray)
+ throws CharacterCodingException {
+
+ String filterName = Bytes.toString(getFilterName(filterStringAsByteArray));
+ ArrayList<byte []> filterArguments = getFilterArguments(filterStringAsByteArray);
+ if (!filterHashMap.containsKey(filterName)) {
+ throw new IllegalArgumentException("Filter Name " + filterName + " not supported");
+ }
+ try {
+ filterName = filterHashMap.get(filterName);
+ Class c = Class.forName(filterName);
+ Class[] argTypes = new Class [] {ArrayList.class};
+ Method m = c.getDeclaredMethod("createFilterFromArguments", argTypes);
+ return (Filter) m.invoke(null,filterArguments);
+ } catch (ClassNotFoundException e) {
+ e.printStackTrace();
+ } catch (NoSuchMethodException e) {
+ e.printStackTrace();
+ } catch (IllegalAccessException e) {
+ e.printStackTrace();
+ } catch (InvocationTargetException e) {
+ e.printStackTrace();
+ }
+ throw new IllegalArgumentException("Incorrect filter string " +
+ new String(filterStringAsByteArray));
+ }
+
+/**
+ * Returns the filter name given a simple filter expression
+ * <p>
+ * @param filterStringAsByteArray a simple filter expression
+ * @return name of filter in the simple filter expression
+ */
+ public static byte [] getFilterName (byte [] filterStringAsByteArray) {
+ int filterNameStartIndex = 0;
+ int filterNameEndIndex = 0;
+
+ for (int i=filterNameStartIndex; i<filterStringAsByteArray.length; i++) {
+ if (filterStringAsByteArray[i] == ParseConstants.LPAREN ||
+ filterStringAsByteArray[i] == ParseConstants.WHITESPACE) {
+ filterNameEndIndex = i;
+ break;
+ }
+ }
+
+ if (filterNameEndIndex == 0) {
+ throw new IllegalArgumentException("Incorrect Filter Name");
+ }
+
+ byte [] filterName = new byte[filterNameEndIndex - filterNameStartIndex];
+ Bytes.putBytes(filterName, 0, filterStringAsByteArray, 0,
+ filterNameEndIndex - filterNameStartIndex);
+ return filterName;
+ }
+
+/**
+ * Returns the arguments of the filter from the filter string
+ * <p>
+ * @param filterStringAsByteArray 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
+ */
+ 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 Unquoted byte array
+ */
+ 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 Bytes#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 Bytes#SIZEOF_INT} 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 ByteArrayComparable createComparator (byte [] comparator) {
+ if (comparator == null)
+ throw new IllegalArgumentException("Incorrect Comparator");
+ byte [][] parsedComparator = ParseFilter.parseComparator(comparator);
+ byte [] comparatorType = parsedComparator[0];
+ byte [] comparatorValue = parsedComparator[1];
+
+
+ if (Bytes.equals(comparatorType, ParseConstants.binaryType))
+ return new BinaryComparator(comparatorValue);
+ else if (Bytes.equals(comparatorType, ParseConstants.binaryPrefixType))
+ return new BinaryPrefixComparator(comparatorValue);
+ else if (Bytes.equals(comparatorType, ParseConstants.regexStringType))
+ return new RegexStringComparator(new String(comparatorValue));
+ else if (Bytes.equals(comparatorType, ParseConstants.substringType))
+ return new SubstringComparator(new String(comparatorValue));
+ else
+ throw new IllegalArgumentException("Incorrect comparatorType");
+ }
+
+/**
+ * Splits a column in comparatorType:comparatorValue form into separate byte arrays
+ * <p>
+ * @param comparator the comparator
+ * @return the parsed arguments of the comparator as a 2D byte array
+ */
+ public static byte [][] parseComparator (byte [] comparator) {
+ final int index = KeyValue.getDelimiter(comparator, 0, comparator.length, ParseConstants.COLON);
+ if (index == -1) {
+ throw new IllegalArgumentException("Incorrect comparator");
+ }
+
+ byte [][] result = new byte [2][0];
+ result[0] = new byte [index];
+ System.arraycopy(comparator, 0, result[0], 0, index);
+
+ final int len = comparator.length - (index + 1);
+ result[1] = new byte[len];
+ System.arraycopy(comparator, index + 1, result[1], 0, len);
+
+ return result;
+ }
+
+/**
+ * Return a Set of filters supported by the Filter Language
+ */
+ public Set<String> getSupportedFilters () {
+ return filterHashMap.keySet();
+ }
+
+ /**
+ * Returns all known filters
+ * @return an unmodifiable map of filters
+ */
+ public static Map<String, String> getAllFilters() {
+ return Collections.unmodifiableMap(filterHashMap);
+ }
+
+ /**
+ * Register a new filter with the parser. If the filter is already registered,
+ * an IllegalArgumentException will be thrown.
+ *
+ * @param name a name for the filter
+ * @param filterClass fully qualified class name
+ */
+ public static void registerFilter(String name, String filterClass) {
+ if(LOG.isInfoEnabled())
+ LOG.info("Registering new filter " + name);
+
+ filterHashMap.put(name, filterClass);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,121 @@
+/*
+ *
+ * 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 com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.ArrayList;
+
+/**
+ * Pass results that have same row prefix.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class PrefixFilter extends FilterBase {
+ protected byte [] prefix = null;
+ protected boolean passedPrefix = false;
+
+ public PrefixFilter(final byte [] prefix) {
+ this.prefix = prefix;
+ }
+
+ public byte[] getPrefix() {
+ return prefix;
+ }
+
+ public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ if (buffer == null || this.prefix == null)
+ return true;
+ if (length < prefix.length)
+ return true;
+ // if they are equal, return false => pass row
+ // else return true, filter row
+ // if we are passed the prefix, set flag
+ int cmp = Bytes.compareTo(buffer, offset, this.prefix.length, this.prefix, 0,
+ this.prefix.length);
+ if(cmp > 0) {
+ passedPrefix = true;
+ }
+ return cmp != 0;
+ }
+
+ public boolean filterAllRemaining() {
+ 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);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.PrefixFilter.Builder builder =
+ FilterProtos.PrefixFilter.newBuilder();
+ if (this.prefix != null) builder.setPrefix(ByteString.copyFrom(this.prefix));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link PrefixFilter} instance
+ * @return An instance of {@link PrefixFilter} made from <code>bytes</code>
+ * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+ * @see #toByteArray
+ */
+ public static PrefixFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.PrefixFilter proto;
+ try {
+ proto = FilterProtos.PrefixFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new PrefixFilter(proto.hasPrefix()?proto.getPrefix().toByteArray():null);
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof PrefixFilter)) return false;
+
+ PrefixFilter other = (PrefixFilter)o;
+ return Bytes.equals(this.getPrefix(), other.getPrefix());
+ }
+
+ @Override
+ public String toString() {
+ return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,128 @@
+/**
+ *
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.io.IOException;
+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
+ * column qualifier portion of a key.
+ * <p>
+ * This filter can be wrapped with {@link WhileMatchFilter} and {@link SkipFilter}
+ * to add more control.
+ * <p>
+ * Multiple filters can be combined using {@link FilterList}.
+ * <p>
+ * If an already known column qualifier is looked for, use {@link Get#addColumn}
+ * directly rather than a filter.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class QualifierFilter extends CompareFilter {
+
+ /**
+ * Constructor.
+ * @param op the compare op for column qualifier matching
+ * @param qualifierComparator the comparator for column qualifier matching
+ */
+ public QualifierFilter(final CompareOp op,
+ final ByteArrayComparable qualifierComparator) {
+ super(op, qualifierComparator);
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ int qualifierLength = v.getQualifierLength();
+ if (qualifierLength > 0) {
+ if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+ v.getQualifierOffset(), qualifierLength)) {
+ return ReturnCode.SKIP;
+ }
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
+ return new QualifierFilter(compareOp, comparator);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.QualifierFilter.Builder builder =
+ FilterProtos.QualifierFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link QualifierFilter} instance
+ * @return An instance of {@link QualifierFilter} made from <code>bytes</code>
+ * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+ * @see #toByteArray
+ */
+ public static QualifierFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.QualifierFilter proto;
+ try {
+ proto = FilterProtos.QualifierFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ final CompareOp valueCompareOp =
+ CompareOp.valueOf(proto.getCompareFilter().getCompareOp().name());
+ ByteArrayComparable valueComparator = null;
+ try {
+ if (proto.getCompareFilter().hasComparator()) {
+ valueComparator = ProtobufUtil.toComparator(proto.getCompareFilter().getComparator());
+ }
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ return new QualifierFilter(valueCompareOp,valueComparator);
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof QualifierFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,149 @@
+/**
+ *
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.util.Random;
+
+/**
+ * A filter that includes rows based on a chance.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class RandomRowFilter extends FilterBase {
+ protected static final Random random = new Random();
+
+ protected float chance;
+ protected boolean filterOutRow;
+
+ /**
+ * Create a new filter with a specified chance for a row to be included.
+ *
+ * @param chance
+ */
+ public RandomRowFilter(float chance) {
+ this.chance = chance;
+ }
+
+ /**
+ * @return The chance that a row gets included.
+ */
+ public float getChance() {
+ return chance;
+ }
+
+ /**
+ * Set the chance that a row is included.
+ *
+ * @param chance
+ */
+ public void setChance(float chance) {
+ this.chance = chance;
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ return false;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ if (filterOutRow) {
+ return ReturnCode.NEXT_ROW;
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ @Override
+ public boolean filterRow() {
+ return filterOutRow;
+ }
+
+ public boolean hasFilterRow() {
+ return true;
+ }
+
+ @Override
+ public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ if (chance < 0) {
+ // with a zero chance, the rows is always excluded
+ filterOutRow = true;
+ } else if (chance > 1) {
+ // always included
+ filterOutRow = false;
+ } else {
+ // roll the dice
+ filterOutRow = !(random.nextFloat() < chance);
+ }
+ return filterOutRow;
+ }
+
+ @Override
+ public void reset() {
+ filterOutRow = false;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.RandomRowFilter.Builder builder =
+ FilterProtos.RandomRowFilter.newBuilder();
+ builder.setChance(this.chance);
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link RandomRowFilter} instance
+ * @return An instance of {@link RandomRowFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static RandomRowFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.RandomRowFilter proto;
+ try {
+ proto = FilterProtos.RandomRowFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new RandomRowFilter(proto.getChance());
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof RandomRowFilter)) return false;
+
+ RandomRowFilter other = (RandomRowFilter)o;
+ return this.getChance() == other.getChance();
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,172 @@
+/**
+ *
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.nio.charset.Charset;
+import java.nio.charset.IllegalCharsetNameException;
+import java.util.regex.Pattern;
+
+/**
+ * This comparator is for use with {@link CompareFilter} implementations, such
+ * as {@link RowFilter}, {@link QualifierFilter}, and {@link ValueFilter}, for
+ * filtering based on the value of a given column. Use it to test if a given
+ * regular expression matches a cell value in the column.
+ * <p>
+ * Only EQUAL or NOT_EQUAL comparisons are valid with this comparator.
+ * <p>
+ * For example:
+ * <p>
+ * <pre>
+ * ValueFilter vf = new ValueFilter(CompareOp.EQUAL,
+ * new RegexStringComparator(
+ * // v4 IP address
+ * "(((25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9]?)\\.){3,3}" +
+ * "(25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9]?))(\\/[0-9]+)?" +
+ * "|" +
+ * // v6 IP address
+ * "((([\\dA-Fa-f]{1,4}:){7}[\\dA-Fa-f]{1,4})(:([\\d]{1,3}.)" +
+ * "{3}[\\d]{1,3})?)(\\/[0-9]+)?"));
+ * </pre>
+ * <p>
+ * Supports {@link java.util.regex.Pattern} flags as well:
+ * <p>
+ * <pre>
+ * ValueFilter vf = new ValueFilter(CompareOp.EQUAL,
+ * new RegexStringComparator("regex", Pattern.CASE_INSENSITIVE | Pattern.DOTALL));
+ * </pre>
+ * @see java.util.regex.Pattern
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class RegexStringComparator extends ByteArrayComparable {
+
+ private static final Log LOG = LogFactory.getLog(RegexStringComparator.class);
+
+ private Charset charset = HConstants.UTF8_CHARSET;
+
+ private Pattern pattern;
+
+ /**
+ * Constructor
+ * Adds Pattern.DOTALL to the underlying Pattern
+ * @param expr a valid regular expression
+ */
+ public RegexStringComparator(String expr) {
+ this(expr, Pattern.DOTALL);
+ }
+
+ /**
+ * Constructor
+ * @param expr a valid regular expression
+ * @param flags java.util.regex.Pattern flags
+ */
+ public RegexStringComparator(String expr, int flags) {
+ super(Bytes.toBytes(expr));
+ this.pattern = Pattern.compile(expr, flags);
+ }
+
+ /**
+ * Specifies the {@link Charset} to use to convert the row key to a String.
+ * <p>
+ * The row key needs to be converted to a String in order to be matched
+ * against the regular expression. This method controls which charset is
+ * used to do this conversion.
+ * <p>
+ * If the row key is made of arbitrary bytes, the charset {@code ISO-8859-1}
+ * is recommended.
+ * @param charset The charset to use.
+ */
+ public void setCharset(final Charset charset) {
+ this.charset = charset;
+ }
+
+ @Override
+ public int compareTo(byte[] value, int offset, int length) {
+ // Use find() for subsequence match instead of matches() (full sequence
+ // match) to adhere to the principle of least surprise.
+ return pattern.matcher(new String(value, offset, length, charset)).find() ? 0
+ : 1;
+ }
+
+ /**
+ * @return The comparator serialized using pb
+ */
+ public byte [] toByteArray() {
+ ComparatorProtos.RegexStringComparator.Builder builder =
+ ComparatorProtos.RegexStringComparator.newBuilder();
+ builder.setPattern(pattern.toString());
+ builder.setPatternFlags(pattern.flags());
+ builder.setCharset(charset.name());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link RegexStringComparator} instance
+ * @return An instance of {@link RegexStringComparator} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static RegexStringComparator parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ ComparatorProtos.RegexStringComparator proto;
+ try {
+ proto = ComparatorProtos.RegexStringComparator.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ RegexStringComparator comparator =
+ new RegexStringComparator(proto.getPattern(), proto.getPatternFlags());
+ final String charset = proto.getCharset();
+ if (charset.length() > 0) {
+ try {
+ comparator.setCharset(Charset.forName(charset));
+ } catch (IllegalCharsetNameException e) {
+ LOG.error("invalid charset", e);
+ }
+ }
+ return comparator;
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the comparator that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(ByteArrayComparable other) {
+ if (other == this) return true;
+ if (!(other instanceof RegexStringComparator)) return false;
+
+ RegexStringComparator comparator = (RegexStringComparator)other;
+ return super.areSerializedFieldsEqual(comparator)
+ && this.pattern.toString().equals(comparator.pattern.toString())
+ && this.pattern.flags() == comparator.pattern.flags()
+ && this.charset.equals(comparator.charset);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,143 @@
+/**
+ *
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+/**
+ * This filter is used to filter based on the key. It takes an operator
+ * (equal, greater, not equal, etc) and a byte [] comparator for the row,
+ * and column qualifier portions of a key.
+ * <p>
+ * This filter can be wrapped with {@link WhileMatchFilter} to add more control.
+ * <p>
+ * Multiple filters can be combined using {@link FilterList}.
+ * <p>
+ * If an already known row range needs to be scanned, use {@link Scan} start
+ * and stop rows directly rather than a filter.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class RowFilter extends CompareFilter {
+
+ private boolean filterOutRow = false;
+
+ /**
+ * Constructor.
+ * @param rowCompareOp the compare op for row matching
+ * @param rowComparator the comparator for row matching
+ */
+ public RowFilter(final CompareOp rowCompareOp,
+ final ByteArrayComparable rowComparator) {
+ super(rowCompareOp, rowComparator);
+ }
+
+ @Override
+ public void reset() {
+ this.filterOutRow = false;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ if(this.filterOutRow) {
+ return ReturnCode.NEXT_ROW;
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ @Override
+ public boolean filterRowKey(byte[] data, int offset, int length) {
+ if(doCompare(this.compareOp, this.comparator, data, offset, length)) {
+ this.filterOutRow = true;
+ }
+ return this.filterOutRow;
+ }
+
+ @Override
+ public boolean filterRow() {
+ return this.filterOutRow;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
+ return new RowFilter(compareOp, comparator);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.RowFilter.Builder builder =
+ FilterProtos.RowFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link RowFilter} instance
+ * @return An instance of {@link RowFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static RowFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.RowFilter proto;
+ try {
+ proto = FilterProtos.RowFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ final CompareOp valueCompareOp =
+ CompareOp.valueOf(proto.getCompareFilter().getCompareOp().name());
+ ByteArrayComparable valueComparator = null;
+ try {
+ if (proto.getCompareFilter().hasComparator()) {
+ valueComparator = ProtobufUtil.toComparator(proto.getCompareFilter().getComparator());
+ }
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ return new RowFilter(valueCompareOp,valueComparator);
+ }
+
+ /**
+ * @param other
+ * @return true if and only if the fields of the filter that are serialized
+ * are equal to the corresponding fields in other. Used for testing.
+ */
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof RowFilter)) return false;
+
+ return super.areSerializedFieldsEqual(o);
+ }
+}