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 [17/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/DependentColumnFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,290 @@
+/**
+ *
+ * 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.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * A filter for adding inter-column timestamp matching
+ * Only cells with a correspondingly timestamped entry in
+ * the target column will be retained
+ * Not compatible with Scan.setBatch as operations need
+ * full rows for correct filtering
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class DependentColumnFilter extends CompareFilter {
+
+ protected byte[] columnFamily;
+ protected byte[] columnQualifier;
+ protected boolean dropDependentColumn;
+
+ protected Set<Long> stampSet = new HashSet<Long>();
+
+ /**
+ * Build a dependent column filter with value checking
+ * dependent column varies will be compared using the supplied
+ * compareOp and comparator, for usage of which
+ * refer to {@link CompareFilter}
+ *
+ * @param family dependent column family
+ * @param qualifier dependent column qualifier
+ * @param dropDependentColumn whether the column should be discarded after
+ * @param valueCompareOp comparison op
+ * @param valueComparator comparator
+ */
+ public DependentColumnFilter(final byte [] family, final byte[] qualifier,
+ final boolean dropDependentColumn, final CompareOp valueCompareOp,
+ final ByteArrayComparable valueComparator) {
+ // set up the comparator
+ super(valueCompareOp, valueComparator);
+ this.columnFamily = family;
+ this.columnQualifier = qualifier;
+ this.dropDependentColumn = dropDependentColumn;
+ }
+
+ /**
+ * Constructor for DependentColumn filter.
+ * Keyvalues where a keyvalue from target column
+ * with the same timestamp do not exist will be dropped.
+ *
+ * @param family name of target column family
+ * @param qualifier name of column qualifier
+ */
+ public DependentColumnFilter(final byte [] family, final byte [] qualifier) {
+ this(family, qualifier, false);
+ }
+
+ /**
+ * Constructor for DependentColumn filter.
+ * Keyvalues where a keyvalue from target column
+ * with the same timestamp do not exist will be dropped.
+ *
+ * @param family name of dependent column family
+ * @param qualifier name of dependent qualifier
+ * @param dropDependentColumn whether the dependent columns keyvalues should be discarded
+ */
+ public DependentColumnFilter(final byte [] family, final byte [] qualifier,
+ final boolean dropDependentColumn) {
+ this(family, qualifier, dropDependentColumn, CompareOp.NO_OP, null);
+ }
+
+ /**
+ * @return the column family
+ */
+ public byte[] getFamily() {
+ return this.columnFamily;
+ }
+
+ /**
+ * @return the column qualifier
+ */
+ public byte[] getQualifier() {
+ return this.columnQualifier;
+ }
+
+ /**
+ * @return true if we should drop the dependent column, false otherwise
+ */
+ public boolean dropDependentColumn() {
+ return this.dropDependentColumn;
+ }
+
+ public boolean getDropDependentColumn() {
+ return this.dropDependentColumn;
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ return false;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ // Check if the column and qualifier match
+ if (!v.matchingColumn(this.columnFamily, this.columnQualifier)) {
+ // include non-matches for the time being, they'll be discarded afterwards
+ return ReturnCode.INCLUDE;
+ }
+ // If it doesn't pass the op, skip it
+ if (comparator != null
+ && doCompare(compareOp, comparator, v.getBuffer(), v.getValueOffset(),
+ v.getValueLength()))
+ return ReturnCode.SKIP;
+
+ stampSet.add(v.getTimestamp());
+ if(dropDependentColumn) {
+ return ReturnCode.SKIP;
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ @Override
+ public void filterRow(List<KeyValue> kvs) {
+ Iterator<KeyValue> it = kvs.iterator();
+ KeyValue kv;
+ while(it.hasNext()) {
+ kv = it.next();
+ if(!stampSet.contains(kv.getTimestamp())) {
+ it.remove();
+ }
+ }
+ }
+
+ @Override
+ public boolean hasFilterRow() {
+ return true;
+ }
+
+ @Override
+ public boolean filterRow() {
+ return false;
+ }
+
+ @Override
+ 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));
+ ByteArrayComparable 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());
+ }
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.DependentColumnFilter.Builder builder =
+ FilterProtos.DependentColumnFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ if (this.columnFamily != null) {
+ builder.setColumnFamily(ByteString.copyFrom(this.columnFamily));
+ }
+ if (this.columnQualifier != null) {
+ builder.setColumnQualifier(ByteString.copyFrom(this.columnQualifier));
+ }
+ builder.setDropDependentColumn(this.dropDependentColumn);
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link DependentColumnFilter} instance
+ * @return An instance of {@link DependentColumnFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static DependentColumnFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.DependentColumnFilter proto;
+ try {
+ proto = FilterProtos.DependentColumnFilter.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 DependentColumnFilter(
+ proto.hasColumnFamily()?proto.getColumnFamily().toByteArray():null,
+ proto.hasColumnQualifier()?proto.getColumnQualifier().toByteArray():null,
+ proto.getDropDependentColumn(), valueCompareOp, valueComparator);
+ }
+
+ /**
+ * @param o
+ * @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.
+ */
+ @edu.umd.cs.findbugs.annotations.SuppressWarnings(
+ value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
+ boolean areSerializedFieldsEqual(Filter o) {
+ if (o == this) return true;
+ if (!(o instanceof DependentColumnFilter)) return false;
+
+ DependentColumnFilter other = (DependentColumnFilter)o;
+ return other != null && super.areSerializedFieldsEqual(other)
+ && Bytes.equals(this.getFamily(), other.getFamily())
+ && Bytes.equals(this.getQualifier(), other.getQualifier())
+ && this.dropDependentColumn() == other.dropDependentColumn();
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s (%s, %s, %s, %s, %s)",
+ this.getClass().getSimpleName(),
+ Bytes.toStringBinary(this.columnFamily),
+ Bytes.toStringBinary(this.columnQualifier),
+ this.dropDependentColumn,
+ this.compareOp.name(),
+ Bytes.toStringBinary(this.comparator.getValue()));
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,129 @@
+/**
+ *
+ * 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.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 family. It takes an
+ * operator (equal, greater, not equal, etc) and a byte [] comparator for the
+ * column family portion of a key.
+ * <p/>
+ * This filter can be wrapped with {@link org.apache.hadoop.hbase.filter.WhileMatchFilter} and {@link org.apache.hadoop.hbase.filter.SkipFilter}
+ * to add more control.
+ * <p/>
+ * Multiple filters can be combined using {@link org.apache.hadoop.hbase.filter.FilterList}.
+ * <p/>
+ * If an already known column family is looked for, use {@link org.apache.hadoop.hbase.client.Get#addFamily(byte[])}
+ * directly rather than a filter.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FamilyFilter extends CompareFilter {
+
+ /**
+ * Constructor.
+ *
+ * @param familyCompareOp the compare op for column family matching
+ * @param familyComparator the comparator for column family matching
+ */
+ public FamilyFilter(final CompareOp familyCompareOp,
+ final ByteArrayComparable familyComparator) {
+ super(familyCompareOp, familyComparator);
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ int familyLength = v.getFamilyLength();
+ if (familyLength > 0) {
+ if (doCompare(this.compareOp, this.comparator, v.getBuffer(),
+ v.getFamilyOffset(), familyLength)) {
+ return ReturnCode.SKIP;
+ }
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ ArrayList arguments = CompareFilter.extractArguments(filterArguments);
+ CompareOp compareOp = (CompareOp)arguments.get(0);
+ ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1);
+ return new FamilyFilter(compareOp, comparator);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FamilyFilter.Builder builder =
+ FilterProtos.FamilyFilter.newBuilder();
+ builder.setCompareFilter(super.convert());
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FamilyFilter} instance
+ * @return An instance of {@link FamilyFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static FamilyFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.FamilyFilter proto;
+ try {
+ proto = FilterProtos.FamilyFilter.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 FamilyFilter(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 FamilyFilter)) return false;
+
+ FamilyFilter other = (FamilyFilter)o;
+ return super.areSerializedFieldsEqual(other);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/Filter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,204 @@
+/*
+ *
+ * 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 org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+
+import java.util.List;
+
+/**
+ * Interface for row and column filters directly applied within the regionserver.
+ * A filter can expect the following call sequence:
+ *<ul>
+ * <li>{@link #reset()}</li>
+ * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+ * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
+ * if false, we will also call</li>
+ * <li>{@link #filterKeyValue(KeyValue)} -> true to drop this key/value</li>
+ * <li>{@link #filterRow(List)} -> allows directmodification of the final list to be submitted
+ * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+ * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+ * </li>
+ * </ul>
+ *
+ * Filter instances are created one per region/scan. This abstract class replaces
+ * the old RowFilterInterface.
+ *
+ * When implementing your own filters, consider inheriting {@link FilterBase} to help
+ * you reduce boilerplate.
+ *
+ * @see FilterBase
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class Filter {
+ /**
+ * Reset the state of the filter between rows.
+ */
+ abstract public void reset();
+
+ /**
+ * Filters a row based on the row key. If this returns true, the entire
+ * row will be excluded. If false, each KeyValue in the row will be
+ * passed to {@link #filterKeyValue(KeyValue)} below.
+ *
+ * @param buffer buffer containing row key
+ * @param offset offset into buffer where row key starts
+ * @param length length of the row key
+ * @return true, remove entire row, false, include the row (maybe).
+ */
+ abstract public boolean filterRowKey(byte [] buffer, int offset, int length);
+
+ /**
+ * If this returns true, the scan will terminate.
+ *
+ * @return true to end scan, false to continue.
+ */
+ abstract public boolean filterAllRemaining();
+
+ /**
+ * A way to filter based on the column family, column qualifier and/or the
+ * column value. Return code is described below. This allows filters to
+ * filter only certain number of columns, then terminate without matching ever
+ * column.
+ *
+ * If your filter returns <code>ReturnCode.NEXT_ROW</code>, it should return
+ * <code>ReturnCode.NEXT_ROW</code> until {@link #reset()} is called
+ * just in case the caller calls for the next row.
+ *
+ * @param v the KeyValue in question
+ * @return code as described below
+ * @see Filter.ReturnCode
+ */
+ abstract public ReturnCode filterKeyValue(final KeyValue v);
+
+ /**
+ * Give the filter a chance to transform the passed KeyValue.
+ * If the KeyValue is changed a new KeyValue object must be returned.
+ * @see org.apache.hadoop.hbase.KeyValue#shallowCopy()
+ *
+ * The transformed KeyValue is what is eventually returned to the
+ * client. Most filters will return the passed KeyValue unchanged.
+ * @see org.apache.hadoop.hbase.filter.KeyOnlyFilter#transform(KeyValue)
+ * for an example of a transformation.
+ *
+ * @param v the KeyValue in question
+ * @return the changed KeyValue
+ */
+ abstract public KeyValue transform(final KeyValue v);
+
+ /**
+ * Return codes for filterValue().
+ */
+ public enum ReturnCode {
+ /**
+ * Include the KeyValue
+ */
+ INCLUDE,
+ /**
+ * Include the KeyValue and seek to the next column skipping older versions.
+ */
+ INCLUDE_AND_NEXT_COL,
+ /**
+ * Skip this KeyValue
+ */
+ SKIP,
+ /**
+ * Skip this column. Go to the next column in this row.
+ */
+ NEXT_COL,
+ /**
+ * Done with columns, skip to next row. Note that filterRow() will
+ * still be called.
+ */
+ NEXT_ROW,
+ /**
+ * Seek to next key which is given as hint by the filter.
+ */
+ SEEK_NEXT_USING_HINT,
+}
+
+ /**
+ * Chance to alter the list of keyvalues to be submitted.
+ * Modifications to the list will carry on
+ * @param kvs the list of keyvalues to be filtered
+ */
+ abstract public void filterRow(List<KeyValue> kvs);
+
+ /**
+ * @return True if this filter actively uses filterRow(List) or filterRow().
+ * Primarily used to check for conflicts with scans(such as scans
+ * that do not read a full row at a time)
+ */
+ abstract public boolean hasFilterRow();
+
+ /**
+ * Last chance to veto row based on previous {@link #filterKeyValue(KeyValue)}
+ * calls. The filter needs to retain state then return a particular value for
+ * this call if they wish to exclude a row if a certain column is missing
+ * (for example).
+ * @return true to exclude row, false to include row.
+ */
+ abstract public boolean filterRow();
+
+ /**
+ * If the filter returns the match code SEEK_NEXT_USING_HINT, then
+ * it should also tell which is the next key it must seek to.
+ * After receiving the match code SEEK_NEXT_USING_HINT, the QueryMatcher would
+ * call this function to find out which key it must next seek to.
+ * @return KeyValue which must be next seeked. return null if the filter is
+ * not sure which key to seek to next.
+ */
+ abstract public KeyValue getNextKeyHint(final KeyValue currentKV);
+
+ /**
+ * Check that given column family is essential for filter to check row. Most
+ * filters always return true here. But some could have more sophisticated
+ * logic which could significantly reduce scanning process by not even
+ * touching columns until we are 100% sure that it's data is needed in result.
+ */
+ abstract public boolean isFamilyEssential(byte[] name);
+
+ /**
+ * @return The filter serialized using pb
+ */
+ abstract public byte [] toByteArray();
+
+ /**
+ * @param pbBytes A pb serialized {@link Filter} instance
+ * @return An instance of {@link Filter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException {
+ throw new DeserializationException(
+ "parseFrom called on base Filter, but should be called on derived type");
+ }
+
+ /**
+ * @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.
+ */
+ abstract boolean areSerializedFieldsEqual(Filter other);
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterBase.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,180 @@
+/*
+ *
+ * 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 org.apache.hadoop.hbase.KeyValue;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Abstract base class to help you implement new Filters. Common "ignore" or NOOP type
+ * methods can go here, helping to reduce boiler plate in an ever-expanding filter
+ * library.
+ *
+ * If you could instantiate FilterBase, it would end up being a "null" filter -
+ * that is one that never filters anything.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class FilterBase extends Filter {
+
+ /**
+ * Filters that are purely stateless and do nothing in their reset() methods can inherit
+ * this null/empty implementation.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public void reset() {
+ }
+
+ /**
+ * Filters that do not filter by row key can inherit this implementation that
+ * never filters anything. (ie: returns false).
+ *
+ * @inheritDoc
+ */
+ @Override
+ public boolean filterRowKey(byte [] buffer, int offset, int length) {
+ return false;
+ }
+
+ /**
+ * Filters that never filter all remaining can inherit this implementation that
+ * never stops the filter early.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public boolean filterAllRemaining() {
+ return false;
+ }
+
+ /**
+ * Filters that dont filter by key value can inherit this implementation that
+ * includes all KeyValues.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public ReturnCode filterKeyValue(KeyValue ignored) {
+ return ReturnCode.INCLUDE;
+ }
+
+ /**
+ * By default no transformation takes place
+ *
+ * @inheritDoc
+ */
+ @Override
+ public KeyValue transform(KeyValue v) {
+ return v;
+ }
+
+ /**
+ * Filters that never filter by modifying the returned List of KeyValues can
+ * inherit this implementation that does nothing.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public void filterRow(List<KeyValue> ignored) {
+ }
+
+ /**
+ * Fitlers that never filter by modifying the returned List of KeyValues can
+ * inherit this implementation that does nothing.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public boolean hasFilterRow() {
+ return false;
+ }
+
+ /**
+ * Filters that never filter by rows based on previously gathered state from
+ * {@link #filterKeyValue(KeyValue)} can inherit this implementation that
+ * never filters a row.
+ *
+ * @inheritDoc
+ */
+ @Override
+ public boolean filterRow() {
+ return false;
+ }
+
+ /**
+ * Filters that are not sure which key must be next seeked to, can inherit
+ * this implementation that, by default, returns a null KeyValue.
+ *
+ * @inheritDoc
+ */
+ public KeyValue getNextKeyHint(KeyValue currentKV) {
+ return null;
+ }
+
+ /**
+ * By default, we require all scan's column families to be present. Our
+ * subclasses may be more precise.
+ *
+ * @inheritDoc
+ */
+ public boolean isFamilyEssential(byte[] name) {
+ return true;
+ }
+
+ /**
+ * 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");
+ }
+
+ /**
+ * Return filter's info for debugging and logging purpose.
+ */
+ public String toString() {
+ return this.getClass().getSimpleName();
+ }
+
+ /**
+ * Return length 0 byte array for Filters that don't require special serialization
+ */
+ public byte [] toByteArray() {
+ return new byte[0];
+ }
+
+ /**
+ * Default implementation so that writers of custom filters aren't forced to implement.
+ *
+ * @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 other) {
+ return true;
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,386 @@
+/**
+ *
+ * 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.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Implementation of {@link Filter} that represents an ordered List of Filters
+ * which will be evaluated with a specified boolean operator {@link Operator#MUST_PASS_ALL}
+ * (<code>!AND</code>) or {@link Operator#MUST_PASS_ONE} (<code>!OR</code>).
+ * Since you can use Filter Lists as children of Filter Lists, you can create a
+ * hierarchy of filters to be evaluated.
+ * Defaults to {@link Operator#MUST_PASS_ALL}.
+ * <p>TODO: Fix creation of Configuration on serialization and deserialization.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FilterList extends Filter {
+ /** set operator */
+ public static enum Operator {
+ /** !AND */
+ MUST_PASS_ALL,
+ /** !OR */
+ MUST_PASS_ONE
+ }
+
+ private static final Configuration conf = HBaseConfiguration.create();
+ private static final int MAX_LOG_FILTERS = 5;
+ private Operator operator = Operator.MUST_PASS_ALL;
+ private List<Filter> filters = new ArrayList<Filter>();
+
+ /**
+ * Constructor that takes a set of {@link Filter}s. The default operator
+ * MUST_PASS_ALL is assumed.
+ *
+ * @param rowFilters list of filters
+ */
+ public FilterList(final List<Filter> rowFilters) {
+ if (rowFilters instanceof ArrayList) {
+ this.filters = rowFilters;
+ } else {
+ this.filters = new ArrayList<Filter>(rowFilters);
+ }
+ }
+
+ /**
+ * Constructor that takes a var arg number of {@link Filter}s. The fefault operator
+ * MUST_PASS_ALL is assumed.
+ * @param rowFilters
+ */
+ public FilterList(final Filter... rowFilters) {
+ this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
+ }
+
+ /**
+ * Constructor that takes an operator.
+ *
+ * @param operator Operator to process filter set with.
+ */
+ public FilterList(final Operator operator) {
+ this.operator = operator;
+ }
+
+ /**
+ * Constructor that takes a set of {@link Filter}s and an operator.
+ *
+ * @param operator Operator to process filter set with.
+ * @param rowFilters Set of row filters.
+ */
+ public FilterList(final Operator operator, final List<Filter> rowFilters) {
+ this.filters = new ArrayList<Filter>(rowFilters);
+ this.operator = operator;
+ }
+
+ /**
+ * Constructor that takes a var arg number of {@link Filter}s and an operator.
+ *
+ * @param operator Operator to process filter set with.
+ * @param rowFilters Filters to use
+ */
+ public FilterList(final Operator operator, final Filter... rowFilters) {
+ this.filters = new ArrayList<Filter>(Arrays.asList(rowFilters));
+ this.operator = operator;
+ }
+
+ /**
+ * Get the operator.
+ *
+ * @return operator
+ */
+ public Operator getOperator() {
+ return operator;
+ }
+
+ /**
+ * Get the filters.
+ *
+ * @return filters
+ */
+ public List<Filter> getFilters() {
+ return filters;
+ }
+
+ /**
+ * Add a filter.
+ *
+ * @param filter another filter
+ */
+ public void addFilter(Filter filter) {
+ this.filters.add(filter);
+ }
+
+ @Override
+ public void reset() {
+ for (Filter filter : filters) {
+ filter.reset();
+ }
+ }
+
+ @Override
+ public boolean filterRowKey(byte[] rowKey, int offset, int length) {
+ for (Filter filter : filters) {
+ if (this.operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining() ||
+ filter.filterRowKey(rowKey, offset, length)) {
+ return true;
+ }
+ } else if (this.operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining() &&
+ !filter.filterRowKey(rowKey, offset, length)) {
+ return false;
+ }
+ }
+ }
+ return this.operator == Operator.MUST_PASS_ONE;
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ for (Filter filter : filters) {
+ if (filter.filterAllRemaining()) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ return true;
+ }
+ } else {
+ if (operator == Operator.MUST_PASS_ONE) {
+ return false;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE;
+ }
+
+ @Override
+ public KeyValue transform(KeyValue v) {
+ KeyValue current = v;
+ for (Filter filter : filters) {
+ current = filter.transform(current);
+ }
+ return current;
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ ReturnCode rc = operator == Operator.MUST_PASS_ONE?
+ ReturnCode.SKIP: ReturnCode.INCLUDE;
+ for (Filter filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining()) {
+ return ReturnCode.NEXT_ROW;
+ }
+ ReturnCode code = filter.filterKeyValue(v);
+ switch (code) {
+ // Override INCLUDE and continue to evaluate.
+ case INCLUDE_AND_NEXT_COL:
+ rc = ReturnCode.INCLUDE_AND_NEXT_COL;
+ case INCLUDE:
+ continue;
+ default:
+ return code;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (filter.filterAllRemaining()) {
+ continue;
+ }
+
+ switch (filter.filterKeyValue(v)) {
+ case INCLUDE:
+ if (rc != ReturnCode.INCLUDE_AND_NEXT_COL) {
+ rc = ReturnCode.INCLUDE;
+ }
+ break;
+ case INCLUDE_AND_NEXT_COL:
+ rc = ReturnCode.INCLUDE_AND_NEXT_COL;
+ // must continue here to evaluate all filters
+ break;
+ case NEXT_ROW:
+ break;
+ case SKIP:
+ // continue;
+ break;
+ case NEXT_COL:
+ break;
+ case SEEK_NEXT_USING_HINT:
+ break;
+ default:
+ throw new IllegalStateException("Received code is not valid.");
+ }
+ }
+ }
+ return rc;
+ }
+
+ @Override
+ public void filterRow(List<KeyValue> kvs) {
+ for (Filter filter : filters) {
+ filter.filterRow(kvs);
+ }
+ }
+
+ @Override
+ public boolean hasFilterRow() {
+ for (Filter filter : filters) {
+ if(filter.hasFilterRow()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public boolean filterRow() {
+ for (Filter filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterRow()) {
+ return true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterRow()) {
+ return false;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FilterList.Builder builder =
+ FilterProtos.FilterList.newBuilder();
+ builder.setOperator(FilterProtos.FilterList.Operator.valueOf(operator.name()));
+ for (Filter filter : filters) {
+ builder.addFilters(ProtobufUtil.toFilter(filter));
+ }
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FilterList} instance
+ * @return An instance of {@link FilterList} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static FilterList parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.FilterList proto;
+ try {
+ proto = FilterProtos.FilterList.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ List<Filter> rowFilters = new ArrayList<Filter>(proto.getFiltersCount());
+ try {
+ for (HBaseProtos.Filter filter : proto.getFiltersList()) {
+ rowFilters.add(ProtobufUtil.toFilter(filter));
+ }
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ return new FilterList(Operator.valueOf(proto.getOperator().name()),rowFilters);
+ }
+
+ /**
+ * @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 FilterList)) return false;
+
+ FilterList other = (FilterList)o;
+ return this.getOperator().equals(other.getOperator()) &&
+ ((this.getFilters() == other.getFilters())
+ || this.getFilters().equals(other.getFilters()));
+ }
+
+ @Override
+ public KeyValue getNextKeyHint(KeyValue currentKV) {
+ KeyValue keyHint = null;
+ for (Filter filter : filters) {
+ KeyValue curKeyHint = filter.getNextKeyHint(currentKV);
+ if (curKeyHint == null && operator == Operator.MUST_PASS_ONE) {
+ // If we ever don't have a hint and this is must-pass-one, then no hint
+ return null;
+ }
+ if (curKeyHint != null) {
+ // If this is the first hint we find, set it
+ if (keyHint == null) {
+ keyHint = curKeyHint;
+ continue;
+ }
+ // There is an existing hint
+ if (operator == Operator.MUST_PASS_ALL &&
+ KeyValue.COMPARATOR.compare(keyHint, curKeyHint) < 0) {
+ // If all conditions must pass, we can keep the max hint
+ keyHint = curKeyHint;
+ } else if (operator == Operator.MUST_PASS_ONE &&
+ KeyValue.COMPARATOR.compare(keyHint, curKeyHint) > 0) {
+ // If any condition can pass, we need to keep the min hint
+ keyHint = curKeyHint;
+ }
+ }
+ }
+ return keyHint;
+ }
+
+ @Override
+ public boolean isFamilyEssential(byte[] name) {
+ for (Filter filter : filters) {
+ if (filter.isFamilyEssential(name)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return toString(MAX_LOG_FILTERS);
+ }
+
+ protected String toString(int maxFilters) {
+ int endIndex = this.filters.size() < maxFilters
+ ? this.filters.size() : maxFilters;
+ return String.format("%s %s (%d/%d): %s",
+ this.getClass().getSimpleName(),
+ this.operator == Operator.MUST_PASS_ALL ? "AND" : "OR",
+ endIndex,
+ this.filters.size(),
+ this.filters.subList(0, endIndex).toString());
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterWrapper.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,154 @@
+/*
+ * Copyright 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 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.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * This is a Filter wrapper class which is used in the server side. Some filter
+ * related hooks can be defined in this wrapper. The only way to create a
+ * FilterWrapper instance is passing a client side Filter instance through
+ * {@link org.apache.hadoop.hbase.client.Scan#getFilter()}.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class FilterWrapper extends Filter {
+ Filter filter = null;
+
+ public FilterWrapper( Filter filter ) {
+ if (null == filter) {
+ // ensure the filter instance is not null
+ throw new NullPointerException("Cannot create FilterWrapper with null Filter");
+ }
+ this.filter = filter;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FilterWrapper.Builder builder =
+ FilterProtos.FilterWrapper.newBuilder();
+ builder.setFilter(ProtobufUtil.toFilter(this.filter));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FilterWrapper} instance
+ * @return An instance of {@link FilterWrapper} made from <code>bytes</code>
+ * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+ * @see #toByteArray
+ */
+ public static FilterWrapper parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.FilterWrapper proto;
+ try {
+ proto = FilterProtos.FilterWrapper.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ try {
+ return new FilterWrapper(ProtobufUtil.toFilter(proto.getFilter()));
+ } catch (IOException ioe) {
+ throw new DeserializationException(ioe);
+ }
+ }
+
+ @Override
+ public void reset() {
+ this.filter.reset();
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ return this.filter.filterAllRemaining();
+ }
+
+ @Override
+ public boolean filterRow() {
+ return this.filter.filterRow();
+ }
+
+ @Override
+ public KeyValue getNextKeyHint(KeyValue currentKV) {
+ return this.filter.getNextKeyHint(currentKV);
+ }
+
+ @Override
+ public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ return this.filter.filterRowKey(buffer, offset, length);
+ }
+
+ @Override
+ public ReturnCode filterKeyValue(KeyValue v) {
+ return this.filter.filterKeyValue(v);
+ }
+
+ @Override
+ public KeyValue transform(KeyValue v) {
+ return this.filter.transform(v);
+ }
+
+ @Override
+ public boolean hasFilterRow() {
+ return this.filter.hasFilterRow();
+ }
+
+ @Override
+ public void filterRow(List<KeyValue> kvs) {
+ //To fix HBASE-6429,
+ //Filter with filterRow() returning true is incompatible with scan with limit
+ //1. hasFilterRow() returns true, if either filterRow() or filterRow(kvs) is implemented.
+ //2. filterRow() is merged with filterRow(kvs),
+ //so that to make all those row related filtering stuff in the same function.
+ this.filter.filterRow(kvs);
+ if (!kvs.isEmpty() && this.filter.filterRow()) {
+ kvs.clear();
+ }
+ }
+
+ @Override
+ public boolean isFamilyEssential(byte[] name) {
+ return filter.isFamilyEssential(name);
+ };
+
+ /**
+ * @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 FilterWrapper)) return false;
+
+ FilterWrapper other = (FilterWrapper)o;
+ return this.filter.areSerializedFieldsEqual(other.filter);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,113 @@
+/*
+ *
+ * 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.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.util.ArrayList;
+
+/**
+ * A filter that will only return the first KV from each row.
+ * <p>
+ * This filter can be used to more efficiently perform row count operations.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FirstKeyOnlyFilter extends FilterBase {
+ private boolean foundKV = false;
+
+ public FirstKeyOnlyFilter() {
+ }
+
+ public void reset() {
+ foundKV = false;
+ }
+
+ public ReturnCode filterKeyValue(KeyValue v) {
+ if(foundKV) return ReturnCode.NEXT_ROW;
+ foundKV = true;
+ 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();
+ }
+
+ /**
+ * @return true if first KV has been found.
+ */
+ protected boolean hasFoundKV() {
+ return this.foundKV;
+ }
+
+ /**
+ *
+ * @param value update {@link #foundKV} flag with value.
+ */
+ protected void setFoundKV(boolean value) {
+ this.foundKV = value;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FirstKeyOnlyFilter.Builder builder =
+ FilterProtos.FirstKeyOnlyFilter.newBuilder();
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FirstKeyOnlyFilter} instance
+ * @return An instance of {@link FirstKeyOnlyFilter} made from <code>bytes</code>
+ * @throws org.apache.hadoop.hbase.exceptions.DeserializationException
+ * @see #toByteArray
+ */
+ public static FirstKeyOnlyFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ // There is nothing to deserialize. Why do this at all?
+ try {
+ FilterProtos.FirstKeyOnlyFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ // Just return a new instance.
+ return new FirstKeyOnlyFilter();
+ }
+
+ /**
+ * @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 FirstKeyOnlyFilter)) return false;
+
+ return true;
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.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.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.Set;
+import java.util.TreeSet;
+
+/**
+ * The filter looks for the given columns in KeyValue. Once there is a match for
+ * any one of the columns, it returns ReturnCode.NEXT_ROW for remaining
+ * KeyValues in the row.
+ * <p>
+ * Note : It may emit KVs which do not have the given columns in them, if
+ * these KVs happen to occur before a KV which does have a match. Given this
+ * caveat, this filter is only useful for special cases
+ * like {@link org.apache.hadoop.hbase.mapreduce.RowCounter}.
+ * <p>
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter {
+
+ private Set<byte []> qualifiers;
+
+ /**
+ * Constructor which takes a set of columns. As soon as first KeyValue
+ * matching any of these columns is found, filter moves to next row.
+ *
+ * @param qualifiers the set of columns to me matched.
+ */
+ public FirstKeyValueMatchingQualifiersFilter(Set<byte []> qualifiers) {
+ this.qualifiers = qualifiers;
+ }
+
+ public ReturnCode filterKeyValue(KeyValue v) {
+ if (hasFoundKV()) {
+ return ReturnCode.NEXT_ROW;
+ } else if (hasOneMatchingQualifier(v)) {
+ setFoundKV(true);
+ }
+ return ReturnCode.INCLUDE;
+ }
+
+ private boolean hasOneMatchingQualifier(KeyValue v) {
+ for (byte[] q : qualifiers) {
+ if (v.matchingQualifier(q)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FirstKeyValueMatchingQualifiersFilter.Builder builder =
+ FilterProtos.FirstKeyValueMatchingQualifiersFilter.newBuilder();
+ for (byte[] qualifier : qualifiers) {
+ if (qualifier != null) builder.addQualifiers(ByteString.copyFrom(qualifier));
+ }
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FirstKeyValueMatchingQualifiersFilter} instance
+ * @return An instance of {@link FirstKeyValueMatchingQualifiersFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static FirstKeyValueMatchingQualifiersFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.FirstKeyValueMatchingQualifiersFilter proto;
+ try {
+ proto = FilterProtos.FirstKeyValueMatchingQualifiersFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+
+ TreeSet<byte []> qualifiers = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
+ for (ByteString qualifier : proto.getQualifiersList()) {
+ qualifiers.add(qualifier.toByteArray());
+ }
+ return new FirstKeyValueMatchingQualifiersFilter(qualifiers);
+ }
+
+ /**
+ * @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 FirstKeyValueMatchingQualifiersFilter)) return false;
+
+ FirstKeyValueMatchingQualifiersFilter other = (FirstKeyValueMatchingQualifiersFilter)o;
+ return this.qualifiers.equals(other.qualifiers);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,332 @@
+/**
+ * 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.protobuf.generated.HBaseProtos.BytesBytesPair;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Filters data based on fuzzy row key. Performs fast-forwards during scanning.
+ * It takes pairs (row key, fuzzy info) to match row keys. Where fuzzy info is
+ * a byte array with 0 or 1 as its values:
+ * <ul>
+ * <li>
+ * 0 - means that this byte in provided row key is fixed, i.e. row key's byte at same position
+ * must match
+ * </li>
+ * <li>
+ * 1 - means that this byte in provided row key is NOT fixed, i.e. row key's byte at this
+ * position can be different from the one in provided row key
+ * </li>
+ * </ul>
+ *
+ *
+ * Example:
+ * Let's assume row key format is userId_actionId_year_month. Length of userId is fixed
+ * and is 4, length of actionId is 2 and year and month are 4 and 2 bytes long respectively.
+ *
+ * Let's assume that we need to fetch all users that performed certain action (encoded as "99")
+ * in Jan of any year. Then the pair (row key, fuzzy info) would be the following:
+ * row key = "????_99_????_01" (one can use any value instead of "?")
+ * fuzzy info = "\x01\x01\x01\x01\x00\x00\x00\x00\x01\x01\x01\x01\x00\x00\x00"
+ *
+ * I.e. fuzzy info tells the matching mask is "????_99_????_01", where at ? can be any value.
+ *
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class FuzzyRowFilter extends FilterBase {
+ private List<Pair<byte[], byte[]>> fuzzyKeysData;
+ private boolean done = false;
+
+ public FuzzyRowFilter(List<Pair<byte[], byte[]>> fuzzyKeysData) {
+ this.fuzzyKeysData = fuzzyKeysData;
+ }
+
+ // TODO: possible improvement: save which fuzzy row key to use when providing a hint
+ @Override
+ public ReturnCode filterKeyValue(KeyValue kv) {
+ byte[] rowKey = kv.getRow();
+ // assigning "worst" result first and looking for better options
+ SatisfiesCode bestOption = SatisfiesCode.NO_NEXT;
+ for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
+ SatisfiesCode satisfiesCode =
+ satisfies(rowKey, fuzzyData.getFirst(), fuzzyData.getSecond());
+ if (satisfiesCode == SatisfiesCode.YES) {
+ return ReturnCode.INCLUDE;
+ }
+
+ if (satisfiesCode == SatisfiesCode.NEXT_EXISTS) {
+ bestOption = SatisfiesCode.NEXT_EXISTS;
+ }
+ }
+
+ if (bestOption == SatisfiesCode.NEXT_EXISTS) {
+ return ReturnCode.SEEK_NEXT_USING_HINT;
+ }
+
+ // the only unhandled SatisfiesCode is NO_NEXT, i.e. we are done
+ done = true;
+ return ReturnCode.NEXT_ROW;
+ }
+
+ @Override
+ public KeyValue getNextKeyHint(KeyValue currentKV) {
+ byte[] rowKey = currentKV.getRow();
+ byte[] nextRowKey = null;
+ // Searching for the "smallest" row key that satisfies at least one fuzzy row key
+ for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
+ byte[] nextRowKeyCandidate = getNextForFuzzyRule(rowKey,
+ fuzzyData.getFirst(), fuzzyData.getSecond());
+ if (nextRowKeyCandidate == null) {
+ continue;
+ }
+ if (nextRowKey == null || Bytes.compareTo(nextRowKeyCandidate, nextRowKey) < 0) {
+ nextRowKey = nextRowKeyCandidate;
+ }
+ }
+
+ if (nextRowKey == null) {
+ // SHOULD NEVER happen
+ // TODO: is there a better way than throw exception? (stop the scanner?)
+ throw new IllegalStateException("No next row key that satisfies fuzzy exists when" +
+ " getNextKeyHint() is invoked." +
+ " Filter: " + this.toString() +
+ " currentKV: " + currentKV.toString());
+ }
+
+ return KeyValue.createFirstOnRow(nextRowKey);
+ }
+
+ @Override
+ public boolean filterAllRemaining() {
+ return done;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.FuzzyRowFilter.Builder builder =
+ FilterProtos.FuzzyRowFilter.newBuilder();
+ for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
+ BytesBytesPair.Builder bbpBuilder = BytesBytesPair.newBuilder();
+ bbpBuilder.setFirst(ByteString.copyFrom(fuzzyData.getFirst()));
+ bbpBuilder.setSecond(ByteString.copyFrom(fuzzyData.getSecond()));
+ builder.addFuzzyKeysData(bbpBuilder);
+ }
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link FuzzyRowFilter} instance
+ * @return An instance of {@link FuzzyRowFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static FuzzyRowFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.FuzzyRowFilter proto;
+ try {
+ proto = FilterProtos.FuzzyRowFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ int count = proto.getFuzzyKeysDataCount();
+ ArrayList<Pair<byte[], byte[]>> fuzzyKeysData= new ArrayList<Pair<byte[], byte[]>>(count);
+ for (int i = 0; i < count; ++i) {
+ BytesBytesPair current = proto.getFuzzyKeysData(i);
+ byte[] keyBytes = current.getFirst().toByteArray();
+ byte[] keyMeta = current.getSecond().toByteArray();
+ fuzzyKeysData.add(new Pair<byte[], byte[]>(keyBytes, keyMeta));
+ }
+ return new FuzzyRowFilter(fuzzyKeysData);
+ }
+
+ @Override
+ public String toString() {
+ final StringBuilder sb = new StringBuilder();
+ sb.append("FuzzyRowFilter");
+ sb.append("{fuzzyKeysData=");
+ for (Pair<byte[], byte[]> fuzzyData : fuzzyKeysData) {
+ sb.append('{').append(Bytes.toStringBinary(fuzzyData.getFirst())).append(":");
+ sb.append(Bytes.toStringBinary(fuzzyData.getSecond())).append('}');
+ }
+ sb.append("}, ");
+ return sb.toString();
+ }
+
+ // Utility methods
+
+ static enum SatisfiesCode {
+ // row satisfies fuzzy rule
+ YES,
+ // row doesn't satisfy fuzzy rule, but there's possible greater row that does
+ NEXT_EXISTS,
+ // row doesn't satisfy fuzzy rule and there's no greater row that does
+ NO_NEXT
+ }
+
+ static SatisfiesCode satisfies(byte[] row,
+ byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ return satisfies(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
+ }
+
+ private static SatisfiesCode satisfies(byte[] row, int offset, int length,
+ byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ if (row == null) {
+ // do nothing, let scan to proceed
+ return SatisfiesCode.YES;
+ }
+
+ boolean nextRowKeyCandidateExists = false;
+
+ for (int i = 0; i < fuzzyKeyMeta.length && i < length; i++) {
+ // First, checking if this position is fixed and not equals the given one
+ boolean byteAtPositionFixed = fuzzyKeyMeta[i] == 0;
+ boolean fixedByteIncorrect = byteAtPositionFixed && fuzzyKeyBytes[i] != row[i + offset];
+ if (fixedByteIncorrect) {
+ // in this case there's another row that satisfies fuzzy rule and bigger than this row
+ if (nextRowKeyCandidateExists) {
+ return SatisfiesCode.NEXT_EXISTS;
+ }
+
+ // If this row byte is less than fixed then there's a byte array bigger than
+ // this row and which satisfies the fuzzy rule. Otherwise there's no such byte array:
+ // this row is simply bigger than any byte array that satisfies the fuzzy rule
+ boolean rowByteLessThanFixed = (row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF);
+ return rowByteLessThanFixed ? SatisfiesCode.NEXT_EXISTS : SatisfiesCode.NO_NEXT;
+ }
+
+ // Second, checking if this position is not fixed and byte value is not the biggest. In this
+ // case there's a byte array bigger than this row and which satisfies the fuzzy rule. To get
+ // bigger byte array that satisfies the rule we need to just increase this byte
+ // (see the code of getNextForFuzzyRule below) by one.
+ // Note: if non-fixed byte is already at biggest value, this doesn't allow us to say there's
+ // bigger one that satisfies the rule as it can't be increased.
+ if (fuzzyKeyMeta[i] == 1 && !isMax(fuzzyKeyBytes[i])) {
+ nextRowKeyCandidateExists = true;
+ }
+ }
+
+ return SatisfiesCode.YES;
+ }
+
+ private static boolean isMax(byte fuzzyKeyByte) {
+ return (fuzzyKeyByte & 0xFF) == 255;
+ }
+
+ static byte[] getNextForFuzzyRule(byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ return getNextForFuzzyRule(row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta);
+ }
+
+ /**
+ * @return greater byte array than given (row) which satisfies the fuzzy rule if it exists,
+ * null otherwise
+ */
+ private static byte[] getNextForFuzzyRule(byte[] row, int offset, int length,
+ byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) {
+ // To find out the next "smallest" byte array that satisfies fuzzy rule and "greater" than
+ // the given one we do the following:
+ // 1. setting values on all "fixed" positions to the values from fuzzyKeyBytes
+ // 2. if during the first step given row did not increase, then we increase the value at
+ // the first "non-fixed" position (where it is not maximum already)
+
+ // It is easier to perform this by using fuzzyKeyBytes copy and setting "non-fixed" position
+ // values than otherwise.
+ byte[] result = Arrays.copyOf(fuzzyKeyBytes,
+ length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length);
+ int toInc = -1;
+
+ boolean increased = false;
+ for (int i = 0; i < result.length; i++) {
+ if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
+ result[i] = row[offset + i];
+ if (!isMax(row[i])) {
+ // this is "non-fixed" position and is not at max value, hence we can increase it
+ toInc = i;
+ }
+ } else if (i < fuzzyKeyMeta.length && fuzzyKeyMeta[i] == 0) {
+ if ((row[i + offset] & 0xFF) < (fuzzyKeyBytes[i] & 0xFF)) {
+ // if setting value for any fixed position increased the original array,
+ // we are OK
+ increased = true;
+ break;
+ }
+ if ((row[i + offset] & 0xFF) > (fuzzyKeyBytes[i] & 0xFF)) {
+ // if setting value for any fixed position makes array "smaller", then just stop:
+ // in case we found some non-fixed position to increase we will do it, otherwise
+ // there's no "next" row key that satisfies fuzzy rule and "greater" than given row
+ break;
+ }
+ }
+ }
+
+ if (!increased) {
+ if (toInc < 0) {
+ return null;
+ }
+ result[toInc]++;
+
+ // Setting all "non-fixed" positions to zeroes to the right of the one we increased so
+ // that found "next" row key is the smallest possible
+ for (int i = toInc + 1; i < result.length; i++) {
+ if (i >= fuzzyKeyMeta.length || fuzzyKeyMeta[i] == 1) {
+ result[i] = 0;
+ }
+ }
+ }
+
+ return result;
+ }
+
+ /**
+ * @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 FuzzyRowFilter)) return false;
+
+ FuzzyRowFilter other = (FuzzyRowFilter)o;
+ if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false;
+ for (int i = 0; i < fuzzyKeysData.size(); ++i) {
+ Pair<byte[], byte[]> thisData = this.fuzzyKeysData.get(i);
+ Pair<byte[], byte[]> otherData = other.fuzzyKeysData.get(i);
+ if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst())
+ && Bytes.equals(thisData.getSecond(), otherData.getSecond()))) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,126 @@
+/*
+ *
+ * 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;
+
+/**
+ * A Filter that stops after the given row. There is no "RowStopFilter" because
+ * the Scan spec allows you to specify a stop row.
+ *
+ * Use this filter to include the stop row, eg: [A,Z].
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InclusiveStopFilter extends FilterBase {
+ private byte [] stopRowKey;
+ private boolean done = false;
+
+ public InclusiveStopFilter(final byte [] stopRowKey) {
+ this.stopRowKey = stopRowKey;
+ }
+
+ public byte[] getStopRowKey() {
+ return this.stopRowKey;
+ }
+
+ public boolean filterRowKey(byte[] buffer, int offset, int length) {
+ if (buffer == null) {
+ //noinspection RedundantIfStatement
+ if (this.stopRowKey == null) {
+ return true; //filter...
+ }
+ return false;
+ }
+ // if stopRowKey is <= buffer, then true, filter row.
+ int cmp = Bytes.compareTo(stopRowKey, 0, stopRowKey.length,
+ buffer, offset, length);
+
+ if(cmp < 0) {
+ done = true;
+ }
+ return done;
+ }
+
+ public boolean filterAllRemaining() {
+ 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);
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.InclusiveStopFilter.Builder builder =
+ FilterProtos.InclusiveStopFilter.newBuilder();
+ if (this.stopRowKey != null) builder.setStopRowKey(ByteString.copyFrom(this.stopRowKey));
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link InclusiveStopFilter} instance
+ * @return An instance of {@link InclusiveStopFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static InclusiveStopFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.InclusiveStopFilter proto;
+ try {
+ proto = FilterProtos.InclusiveStopFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new InclusiveStopFilter(proto.hasStopRowKey()?proto.getStopRowKey().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 InclusiveStopFilter)) return false;
+
+ InclusiveStopFilter other = (InclusiveStopFilter)o;
+ return Bytes.equals(this.getStopRowKey(), other.getStopRowKey());
+ }
+
+ @Override
+ public String toString() {
+ return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/IncompatibleFilterException.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,44 @@
+/**
+ *
+ * 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;
+
+/**
+ * Used to indicate a filter incompatibility
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class IncompatibleFilterException extends RuntimeException {
+ private static final long serialVersionUID = 3236763276623198231L;
+
+/** constructor */
+ public IncompatibleFilterException() {
+ super();
+ }
+
+ /**
+ * constructor
+ * @param s message
+ */
+ public IncompatibleFilterException(String s) {
+ super(s);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,45 @@
+/**
+ *
+ * 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;
+
+/**
+ * Used to indicate an invalid RowFilter.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class InvalidRowFilterException extends RuntimeException {
+ private static final long serialVersionUID = 2667894046345657865L;
+
+
+ /** constructor */
+ public InvalidRowFilterException() {
+ super();
+ }
+
+ /**
+ * constructor
+ * @param s message
+ */
+ public InvalidRowFilterException(String s) {
+ super(s);
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java?rev=1449950&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java Mon Feb 25 22:50:17 2013
@@ -0,0 +1,101 @@
+/*
+ *
+ * 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.KeyValue;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
+
+import java.util.ArrayList;
+
+/**
+ * A filter that will only return the key component of each KV (the value will
+ * be rewritten as empty).
+ * <p>
+ * This filter can be used to grab all of the keys without having to also grab
+ * the values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class KeyOnlyFilter extends FilterBase {
+
+ boolean lenAsVal;
+ public KeyOnlyFilter() { this(false); }
+ public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; }
+
+ @Override
+ public KeyValue transform(KeyValue kv) {
+ return kv.createKeyOnly(this.lenAsVal);
+ }
+
+ public static Filter createFilterFromArguments(ArrayList<byte []> filterArguments) {
+ Preconditions.checkArgument((filterArguments.size() == 0 || filterArguments.size() == 1),
+ "Expected: 0 or 1 but got: %s", filterArguments.size());
+ KeyOnlyFilter filter = new KeyOnlyFilter();
+ if (filterArguments.size() == 1) {
+ filter.lenAsVal = ParseFilter.convertByteArrayToBoolean(filterArguments.get(0));
+ }
+ return filter;
+ }
+
+ /**
+ * @return The filter serialized using pb
+ */
+ public byte [] toByteArray() {
+ FilterProtos.KeyOnlyFilter.Builder builder =
+ FilterProtos.KeyOnlyFilter.newBuilder();
+ builder.setLenAsVal(this.lenAsVal);
+ return builder.build().toByteArray();
+ }
+
+ /**
+ * @param pbBytes A pb serialized {@link KeyOnlyFilter} instance
+ * @return An instance of {@link KeyOnlyFilter} made from <code>bytes</code>
+ * @throws DeserializationException
+ * @see #toByteArray
+ */
+ public static KeyOnlyFilter parseFrom(final byte [] pbBytes)
+ throws DeserializationException {
+ FilterProtos.KeyOnlyFilter proto;
+ try {
+ proto = FilterProtos.KeyOnlyFilter.parseFrom(pbBytes);
+ } catch (InvalidProtocolBufferException e) {
+ throw new DeserializationException(e);
+ }
+ return new KeyOnlyFilter(proto.getLenAsVal());
+ }
+
+ /**
+ * @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 KeyOnlyFilter)) return false;
+
+ KeyOnlyFilter other = (KeyOnlyFilter)o;
+ return this.lenAsVal == other.lenAsVal;
+ }
+}