diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java index 3aaac36268..f22e05d9bf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnCountGetFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -53,7 +54,9 @@ public class ColumnCountGetFilter extends FilterBase { @Override public boolean filterRowKey(Cell cell) throws IOException { // Impl in FilterBase might do unnecessary copy for Off heap backed Cells. - if (filterAllRemaining()) return true; + if (filterAllRemaining()) { + return true; + } return false; } @@ -79,9 +82,8 @@ public class ColumnCountGetFilter extends FilterBase { this.count = 0; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - Preconditions.checkArgument(filterArguments.size() == 1, - "Expected 1 but got: %s", filterArguments.size()); + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 1, "Expected 1 but got: %s", filterArguments.size()); int limit = ParseFilter.convertByteArrayToInt(filterArguments.get(0)); return new ColumnCountGetFilter(limit); } @@ -90,9 +92,8 @@ public class ColumnCountGetFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.ColumnCountGetFilter.Builder builder = - FilterProtos.ColumnCountGetFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.ColumnCountGetFilter.Builder builder = FilterProtos.ColumnCountGetFilter.newBuilder(); builder.setLimit(this.limit); return builder.build().toByteArray(); } @@ -103,8 +104,7 @@ public class ColumnCountGetFilter extends FilterBase { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static ColumnCountGetFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static ColumnCountGetFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.ColumnCountGetFilter proto; try { proto = FilterProtos.ColumnCountGetFilter.parseFrom(pbBytes); @@ -121,10 +121,14 @@ public class ColumnCountGetFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof ColumnCountGetFilter)) return false; + if (o == this) { + return true; + } + if (!(o instanceof ColumnCountGetFilter)) { + return false; + } - ColumnCountGetFilter other = (ColumnCountGetFilter)o; + ColumnCountGetFilter other = (ColumnCountGetFilter) o; return this.getLimit() == other.getLimit(); } @@ -132,4 +136,19 @@ public class ColumnCountGetFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.limit; } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ColumnCountGetFilter))) { + return false; + } + ColumnCountGetFilter f = (ColumnCountGetFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.limit); + } + } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java index c90047da8b..4f592e98ac 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPaginationFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -230,4 +231,18 @@ public class ColumnPaginationFilter extends FilterBase { return String.format("%s (%d, %d)", this.getClass().getSimpleName(), this.limit, this.offset); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ColumnPaginationFilter))) { + return false; + } + ColumnPaginationFilter f = (ColumnPaginationFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.limit, this.offset); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java index a403a40425..7a959ff916 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnPrefixFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.ByteBufferExtendedCell; import org.apache.hadoop.hbase.Cell; @@ -42,9 +43,9 @@ import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; */ @InterfaceAudience.Public public class ColumnPrefixFilter extends FilterBase { - protected byte [] prefix = null; + protected byte[] prefix = null; - public ColumnPrefixFilter(final byte [] prefix) { + public ColumnPrefixFilter(final byte[] prefix) { this.prefix = prefix; } @@ -99,14 +100,12 @@ public class ColumnPrefixFilter extends FilterBase { return ByteBufferUtils.compareTo(((ByteBufferExtendedCell) cell).getQualifierByteBuffer(), ((ByteBufferExtendedCell) cell).getQualifierPosition(), length, prefix, 0, length); } - return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0, - length); + return Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), length, prefix, 0, length); } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - Preconditions.checkArgument(filterArguments.size() == 1, - "Expected 1 but got: %s", filterArguments.size()); - byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 1, "Expected 1 but got: %s", filterArguments.size()); + byte[] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); return new ColumnPrefixFilter(columnPrefix); } @@ -114,10 +113,10 @@ public class ColumnPrefixFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.ColumnPrefixFilter.Builder builder = - FilterProtos.ColumnPrefixFilter.newBuilder(); - if (this.prefix != null) builder.setPrefix(UnsafeByteOperations.unsafeWrap(this.prefix)); + public byte[] toByteArray() { + FilterProtos.ColumnPrefixFilter.Builder builder = FilterProtos.ColumnPrefixFilter.newBuilder(); + if (this.prefix != null) + builder.setPrefix(UnsafeByteOperations.unsafeWrap(this.prefix)); return builder.build().toByteArray(); } @@ -127,8 +126,7 @@ public class ColumnPrefixFilter extends FilterBase { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static ColumnPrefixFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static ColumnPrefixFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.ColumnPrefixFilter proto; try { proto = FilterProtos.ColumnPrefixFilter.parseFrom(pbBytes); @@ -145,10 +143,12 @@ public class ColumnPrefixFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof ColumnPrefixFilter)) return false; + if (o == this) + return true; + if (!(o instanceof ColumnPrefixFilter)) + return false; - ColumnPrefixFilter other = (ColumnPrefixFilter)o; + ColumnPrefixFilter other = (ColumnPrefixFilter) o; return Bytes.equals(this.getPrefix(), other.getPrefix()); } @@ -161,4 +161,18 @@ public class ColumnPrefixFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ColumnPrefixFilter))) { + return false; + } + ColumnPrefixFilter f = (ColumnPrefixFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.getPrefix())); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java index f981ed75e4..d717b99c0c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnRangeFilter.java @@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.util.Bytes.len; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -65,8 +66,8 @@ public class ColumnRangeFilter extends FilterBase { * @param maxColumnInclusive if true, include maxColumn in the range. * there is no upper bound. */ - public ColumnRangeFilter(final byte[] minColumn, boolean minColumnInclusive, - final byte[] maxColumn, boolean maxColumnInclusive) { + public ColumnRangeFilter(final byte[] minColumn, boolean minColumnInclusive, final byte[] maxColumn, + boolean maxColumnInclusive) { this.minColumn = minColumn; this.minColumnInclusive = minColumnInclusive; this.maxColumn = maxColumn; @@ -156,34 +157,33 @@ public class ColumnRangeFilter extends FilterBase { return ReturnCode.NEXT_ROW; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - Preconditions.checkArgument(filterArguments.size() == 4, - "Expected 4 but got: %s", filterArguments.size()); - byte [] minColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.size() == 4, "Expected 4 but got: %s", filterArguments.size()); + byte[] minColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); boolean minColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(1)); - byte [] maxColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(2)); + byte[] maxColumn = ParseFilter.removeQuotesFromByteArray(filterArguments.get(2)); boolean maxColumnInclusive = ParseFilter.convertByteArrayToBoolean(filterArguments.get(3)); if (minColumn.length == 0) minColumn = null; if (maxColumn.length == 0) maxColumn = null; - return new ColumnRangeFilter(minColumn, minColumnInclusive, - maxColumn, maxColumnInclusive); + return new ColumnRangeFilter(minColumn, minColumnInclusive, maxColumn, maxColumnInclusive); } /** * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.ColumnRangeFilter.Builder builder = - FilterProtos.ColumnRangeFilter.newBuilder(); - if (this.minColumn != null) builder.setMinColumn( - UnsafeByteOperations.unsafeWrap(this.minColumn)); + public byte[] toByteArray() { + FilterProtos.ColumnRangeFilter.Builder builder = FilterProtos.ColumnRangeFilter.newBuilder(); + if (this.minColumn != null) { + builder.setMinColumn(UnsafeByteOperations.unsafeWrap(this.minColumn)); + } builder.setMinColumnInclusive(this.minColumnInclusive); - if (this.maxColumn != null) builder.setMaxColumn( - UnsafeByteOperations.unsafeWrap(this.maxColumn)); + if (this.maxColumn != null) { + builder.setMaxColumn(UnsafeByteOperations.unsafeWrap(this.maxColumn)); + } builder.setMaxColumnInclusive(this.maxColumnInclusive); return builder.build().toByteArray(); } @@ -194,17 +194,16 @@ public class ColumnRangeFilter extends FilterBase { * @throws DeserializationException * @see #toByteArray */ - public static ColumnRangeFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static ColumnRangeFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.ColumnRangeFilter proto; try { proto = FilterProtos.ColumnRangeFilter.parseFrom(pbBytes); } catch (InvalidProtocolBufferException e) { throw new DeserializationException(e); } - return new ColumnRangeFilter(proto.hasMinColumn()?proto.getMinColumn().toByteArray():null, - proto.getMinColumnInclusive(),proto.hasMaxColumn()?proto.getMaxColumn().toByteArray():null, - proto.getMaxColumnInclusive()); + return new ColumnRangeFilter(proto.hasMinColumn() ? proto.getMinColumn().toByteArray() : null, + proto.getMinColumnInclusive(), proto.hasMaxColumn() ? proto.getMaxColumn().toByteArray() : null, + proto.getMaxColumnInclusive()); } /** @@ -234,9 +233,23 @@ public class ColumnRangeFilter extends FilterBase { @Override public String toString() { - return this.getClass().getSimpleName() + " " - + (this.minColumnInclusive ? "[" : "(") + Bytes.toStringBinary(this.minColumn) - + ", " + Bytes.toStringBinary(this.maxColumn) + return this.getClass().getSimpleName() + " " + (this.minColumnInclusive ? "[" : "(") + + Bytes.toStringBinary(this.minColumn) + ", " + Bytes.toStringBinary(this.maxColumn) + (this.maxColumnInclusive ? "]" : ")"); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ColumnRangeFilter))) { + return false; + } + ColumnRangeFilter f = (ColumnRangeFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.toStringBinary(this.getMinColumn()), this.getMinColumnInclusive(), + Bytes.toStringBinary(this.getMaxColumn()), this.getMaxColumnInclusive()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java index 07951658f8..9e61dba819 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -238,4 +239,18 @@ public class ColumnValueFilter extends FilterBase { Bytes.toStringBinary(this.qualifier), this.op.name(), Bytes.toStringBinary(this.comparator.getValue())); } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ColumnValueFilter))) { + return false; + } + ColumnValueFilter f = (ColumnValueFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.getFamily()), Bytes.hashCode(this.getQualifier()) + , this.getCompareOperator(), this.getComparator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java index 50924ec631..73c60fd646 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/CompareFilter.java @@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; @@ -321,4 +322,18 @@ public abstract class CompareFilter extends FilterBase { this.op.name(), Bytes.toStringBinary(this.comparator.getValue())); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof CompareFilter))) { + return false; + } + CompareFilter f = (CompareFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getCompareOperator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java index 8f5dee7924..7d8dd60d8d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/DependentColumnFilter.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Objects; import java.util.Set; import org.apache.hadoop.hbase.Cell; @@ -311,4 +312,18 @@ public class DependentColumnFilter extends CompareFilter { this.op.name(), this.comparator != null ? Bytes.toStringBinary(this.comparator.getValue()) : "null"); } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof DependentColumnFilter))) { + return false; + } + DependentColumnFilter f = (DependentColumnFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.getFamily()),Bytes.hashCode(this.getQualifier()), + this.dropDependentColumn()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java index f114e98b7d..57279f0223 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FamilyFilter.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; - +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.yetus.audience.InterfaceAudience; @@ -146,4 +146,18 @@ public class FamilyFilter extends CompareFilter { FamilyFilter other = (FamilyFilter)o; return super.areSerializedFieldsEqual(other); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FamilyFilter))) { + return false; + } + FamilyFilter f = (FamilyFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getCompareOperator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java index c549eabb6c..f8be2f6db9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterList.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -276,4 +277,17 @@ final public class FilterList extends FilterBase { public String toString() { return this.filterListBase.toString(); } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FilterList))) { + return false; + } + FilterList f = (FilterList) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getOperator(), Arrays.hashCode(this.getFilters().toArray())); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java index ebff3a5d73..b9fe00ab79 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithAND.java @@ -24,8 +24,10 @@ import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Objects; /** * FilterListWithAND represents an ordered list of filters which will be evaluated with an AND @@ -279,4 +281,21 @@ public class FilterListWithAND extends FilterListBase { } return maxHint; } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FilterListWithAND))) { + return false; + } + if (this == obj) { + return true; + } + FilterListWithAND f = (FilterListWithAND) obj; + return this.filters.equals(f.getFilters()) + &&this.seekHintFilters.equals(f.seekHintFilters); + } + + @Override + public int hashCode() { + return Objects.hash(this.seekHintFilters, Arrays.hashCode(this.filters.toArray())); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java index 064dd8387b..991563e13c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FilterListWithOR.java @@ -26,8 +26,10 @@ import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Objects; /** * FilterListWithOR represents an ordered list of filters which will be evaluated with an OR @@ -391,4 +393,26 @@ public class FilterListWithOR extends FilterListBase { } return minKeyHint; } + + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FilterListWithOR))) { + return false; + } + if (this == obj) { + return true; + } + FilterListWithOR f = (FilterListWithOR) obj; + return this.filters.equals(f.getFilters()) + &&this.prevFilterRCList.equals(f.prevFilterRCList) + &&this.prevCellList.equals(f.prevCellList); + } + + @Override + public int hashCode() { + return Objects.hash(Arrays.hashCode(this.prevFilterRCList.toArray()), + Arrays.hashCode(this.prevCellList.toArray()), + Arrays.hashCode(this.filters.toArray())); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java index a6b990f4b7..4a7e4356da 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyOnlyFilter.java @@ -60,14 +60,14 @@ public class FirstKeyOnlyFilter extends FilterBase { @Override public ReturnCode filterCell(final Cell c) { - if(foundKV) return ReturnCode.NEXT_ROW; + if (foundKV) + return ReturnCode.NEXT_ROW; foundKV = true; return ReturnCode.INCLUDE; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - Preconditions.checkArgument(filterArguments.isEmpty(), - "Expected 0 but got: %s", filterArguments.size()); + public static Filter createFilterFromArguments(ArrayList filterArguments) { + Preconditions.checkArgument(filterArguments.isEmpty(), "Expected 0 but got: %s", filterArguments.size()); return new FirstKeyOnlyFilter(); } @@ -90,9 +90,8 @@ public class FirstKeyOnlyFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.FirstKeyOnlyFilter.Builder builder = - FilterProtos.FirstKeyOnlyFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.FirstKeyOnlyFilter.Builder builder = FilterProtos.FirstKeyOnlyFilter.newBuilder(); return builder.build().toByteArray(); } @@ -102,8 +101,7 @@ public class FirstKeyOnlyFilter extends FilterBase { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static FirstKeyOnlyFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static FirstKeyOnlyFilter parseFrom(final byte[] pbBytes) throws DeserializationException { // There is nothing to deserialize. Why do this at all? try { FilterProtos.FirstKeyOnlyFilter.parseFrom(pbBytes); @@ -121,8 +119,10 @@ public class FirstKeyOnlyFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof FirstKeyOnlyFilter)) return false; + if (o == this) + return true; + if (!(o instanceof FirstKeyOnlyFilter)) + return false; return true; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java index 88dc36a740..4775643fb3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FirstKeyValueMatchingQualifiersFilter.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.filter; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; @@ -133,4 +134,18 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { FirstKeyValueMatchingQualifiersFilter other = (FirstKeyValueMatchingQualifiersFilter)o; return this.qualifiers.equals(other.qualifiers); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FirstKeyValueMatchingQualifiersFilter))) { + return false; + } + FirstKeyValueMatchingQualifiersFilter f = (FirstKeyValueMatchingQualifiersFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.qualifiers); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java index 714c550ddd..8d17ddcc34 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/FuzzyRowFilter.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.PriorityQueue; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -80,8 +81,8 @@ public class FuzzyRowFilter extends FilterBase { for (Pair aFuzzyKeysData : fuzzyKeysData) { if (aFuzzyKeysData.getFirst().length != aFuzzyKeysData.getSecond().length) { - Pair readable = - new Pair<>(Bytes.toStringBinary(aFuzzyKeysData.getFirst()), Bytes.toStringBinary(aFuzzyKeysData.getSecond())); + Pair readable = new Pair<>(Bytes.toStringBinary(aFuzzyKeysData.getFirst()), + Bytes.toStringBinary(aFuzzyKeysData.getSecond())); throw new IllegalArgumentException("Fuzzy pair lengths do not match: " + readable); } @@ -100,7 +101,6 @@ public class FuzzyRowFilter extends FilterBase { this.tracker = new RowTracker(); } - private void preprocessSearchKey(Pair p) { if (!UNSAFE_UNALIGNED) { // do nothing @@ -127,7 +127,8 @@ public class FuzzyRowFilter extends FilterBase { // do nothing return mask; } - if (isPreprocessedMask(mask)) return mask; + if (isPreprocessedMask(mask)) + return mask; for (int i = 0; i < mask.length; i++) { if (mask[i] == 0) { mask[i] = -1; // 0 -> -1 @@ -164,9 +165,8 @@ public class FuzzyRowFilter extends FilterBase { for (int j = 0; j < fuzzyData.getSecond().length; j++) { fuzzyData.getSecond()[j] >>= 2; } - SatisfiesCode satisfiesCode = - satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(), - fuzzyData.getFirst(), fuzzyData.getSecond()); + SatisfiesCode satisfiesCode = satisfies(isReversed(), c.getRowArray(), c.getRowOffset(), c.getRowLength(), + fuzzyData.getFirst(), fuzzyData.getSecond()); if (satisfiesCode == SatisfiesCode.YES) { lastFoundIndex = index; return ReturnCode.INCLUDE; @@ -203,15 +203,13 @@ public class FuzzyRowFilter extends FilterBase { private boolean initialized = false; RowTracker() { - nextRows = new PriorityQueue<>(fuzzyKeysData.size(), - new Comparator>>() { - @Override - public int compare(Pair> o1, - Pair> o2) { - return isReversed()? Bytes.compareTo(o2.getFirst(), o1.getFirst()): - Bytes.compareTo(o1.getFirst(), o2.getFirst()); - } - }); + nextRows = new PriorityQueue<>(fuzzyKeysData.size(), new Comparator>>() { + @Override + public int compare(Pair> o1, Pair> o2) { + return isReversed() ? Bytes.compareTo(o2.getFirst(), o1.getFirst()) + : Bytes.compareTo(o1.getFirst(), o2.getFirst()); + } + }); } byte[] nextRow() { @@ -245,9 +243,8 @@ public class FuzzyRowFilter extends FilterBase { } void updateWith(Cell currentCell, Pair fuzzyData) { - byte[] nextRowKeyCandidate = - getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), currentCell.getRowOffset(), - currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); + byte[] nextRowKeyCandidate = getNextForFuzzyRule(isReversed(), currentCell.getRowArray(), + currentCell.getRowOffset(), currentCell.getRowLength(), fuzzyData.getFirst(), fuzzyData.getSecond()); if (nextRowKeyCandidate != null) { nextRows.add(new Pair<>(nextRowKeyCandidate, fuzzyData)); } @@ -329,13 +326,12 @@ public class FuzzyRowFilter extends FilterBase { } @VisibleForTesting - static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, - byte[] fuzzyKeyMeta) { + static SatisfiesCode satisfies(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { return satisfies(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); } - static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + static SatisfiesCode satisfies(boolean reverse, byte[] row, int offset, int length, byte[] fuzzyKeyBytes, + byte[] fuzzyKeyMeta) { if (!UNSAFE_UNALIGNED) { return satisfiesNoUnsafe(reverse, row, offset, length, fuzzyKeyBytes, fuzzyKeyMeta); @@ -398,8 +394,8 @@ public class FuzzyRowFilter extends FilterBase { return SatisfiesCode.YES; } - static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + static SatisfiesCode satisfiesNoUnsafe(boolean reverse, byte[] row, int offset, int length, byte[] fuzzyKeyBytes, + byte[] fuzzyKeyMeta) { if (row == null) { // do nothing, let scan to proceed return SatisfiesCode.YES; @@ -450,8 +446,7 @@ public class FuzzyRowFilter extends FilterBase { } @VisibleForTesting - static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, - byte[] fuzzyKeyMeta) { + static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { return getNextForFuzzyRule(reverse, row, 0, row.length, fuzzyKeyBytes, fuzzyKeyMeta); } @@ -537,8 +532,8 @@ public class FuzzyRowFilter extends FilterBase { * otherwise */ @VisibleForTesting - static byte[] getNextForFuzzyRule(boolean reverse, byte[] row, int offset, int length, - byte[] fuzzyKeyBytes, byte[] fuzzyKeyMeta) { + static byte[] getNextForFuzzyRule(boolean reverse, 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 @@ -547,8 +542,7 @@ public class FuzzyRowFilter extends FilterBase { // 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); + byte[] result = Arrays.copyOf(fuzzyKeyBytes, length > fuzzyKeyBytes.length ? length : fuzzyKeyBytes.length); if (reverse && length > fuzzyKeyBytes.length) { // we need trailing 0xff's instead of trailing 0x00's for (int i = fuzzyKeyBytes.length; i < result.length; i++) { @@ -598,7 +592,7 @@ public class FuzzyRowFilter extends FilterBase { } } - return reverse? result: trimTrailingZeroes(result, fuzzyKeyMeta, toInc); + return reverse ? result : trimTrailingZeroes(result, fuzzyKeyMeta, toInc); } /** @@ -612,15 +606,16 @@ public class FuzzyRowFilter extends FilterBase { * @param toInc - position of incremented byte * @return trimmed version of result */ - + private static byte[] trimTrailingZeroes(byte[] result, byte[] fuzzyKeyMeta, int toInc) { - int off = fuzzyKeyMeta.length >= result.length? result.length -1: - fuzzyKeyMeta.length -1; - for( ; off >= 0; off--){ - if(fuzzyKeyMeta[off] != 0) break; + int off = fuzzyKeyMeta.length >= result.length ? result.length - 1 : fuzzyKeyMeta.length - 1; + for (; off >= 0; off--) { + if (fuzzyKeyMeta[off] != 0) + break; } - if (off < toInc) off = toInc; - byte[] retValue = new byte[off+1]; + if (off < toInc) + off = toInc; + byte[] retValue = new byte[off + 1]; System.arraycopy(result, 0, retValue, 0, retValue.length); return retValue; } @@ -631,19 +626,36 @@ public class FuzzyRowFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof FuzzyRowFilter)) return false; + if (o == this) + return true; + if (!(o instanceof FuzzyRowFilter)) + return false; FuzzyRowFilter other = (FuzzyRowFilter) o; - if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) return false; + if (this.fuzzyKeysData.size() != other.fuzzyKeysData.size()) + return false; for (int i = 0; i < fuzzyKeysData.size(); ++i) { Pair thisData = this.fuzzyKeysData.get(i); Pair otherData = other.fuzzyKeysData.get(i); - if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst()) && Bytes.equals( - thisData.getSecond(), otherData.getSecond()))) { + if (!(Bytes.equals(thisData.getFirst(), otherData.getFirst()) + && Bytes.equals(thisData.getSecond(), otherData.getSecond()))) { return false; } } return true; } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof FuzzyRowFilter))) { + return false; + } + FuzzyRowFilter f = (FuzzyRowFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.fuzzyKeysData.toArray()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java index 37564994de..0e1749a3ea 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/InclusiveStopFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -131,4 +132,18 @@ public class InclusiveStopFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof InclusiveStopFilter))) { + return false; + } + InclusiveStopFilter f = (InclusiveStopFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.stopRowKey)); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java index a66441bceb..65c87b98c5 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/KeyOnlyFilter.java @@ -24,6 +24,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.Optional; +import java.util.Objects; + import org.apache.hadoop.hbase.ByteBufferExtendedCell; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; @@ -48,8 +50,14 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; public class KeyOnlyFilter extends FilterBase { boolean lenAsVal; - public KeyOnlyFilter() { this(false); } - public KeyOnlyFilter(boolean lenAsVal) { this.lenAsVal = lenAsVal; } + + public KeyOnlyFilter() { + this(false); + } + + public KeyOnlyFilter(boolean lenAsVal) { + this.lenAsVal = lenAsVal; + } @Override public boolean filterRowKey(Cell cell) throws IOException { @@ -57,6 +65,10 @@ public class KeyOnlyFilter extends FilterBase { return false; } + public boolean isLenAsVal() { + return this.lenAsVal; + } + @Override public Cell transformCell(Cell cell) { return createKeyOnlyCell(cell); @@ -81,9 +93,9 @@ public class KeyOnlyFilter extends FilterBase { return ReturnCode.INCLUDE; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { + public static Filter createFilterFromArguments(ArrayList filterArguments) { Preconditions.checkArgument((filterArguments.isEmpty() || filterArguments.size() == 1), - "Expected: 0 or 1 but got: %s", filterArguments.size()); + "Expected: 0 or 1 but got: %s", filterArguments.size()); KeyOnlyFilter filter = new KeyOnlyFilter(); if (filterArguments.size() == 1) { filter.lenAsVal = ParseFilter.convertByteArrayToBoolean(filterArguments.get(0)); @@ -95,9 +107,8 @@ public class KeyOnlyFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.KeyOnlyFilter.Builder builder = - FilterProtos.KeyOnlyFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.KeyOnlyFilter.Builder builder = FilterProtos.KeyOnlyFilter.newBuilder(); builder.setLenAsVal(this.lenAsVal); return builder.build().toByteArray(); } @@ -108,8 +119,7 @@ public class KeyOnlyFilter extends FilterBase { * @throws DeserializationException * @see #toByteArray */ - public static KeyOnlyFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static KeyOnlyFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.KeyOnlyFilter proto; try { proto = FilterProtos.KeyOnlyFilter.parseFrom(pbBytes); @@ -126,10 +136,12 @@ public class KeyOnlyFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof KeyOnlyFilter)) return false; + if (o == this) + return true; + if (!(o instanceof KeyOnlyFilter)) + return false; - KeyOnlyFilter other = (KeyOnlyFilter)o; + KeyOnlyFilter other = (KeyOnlyFilter) o; return this.lenAsVal == other.lenAsVal; } @@ -202,7 +214,6 @@ public class KeyOnlyFilter extends FilterBase { return cell.getType(); } - @Override public long getSequenceId() { return 0; @@ -247,9 +258,22 @@ public class KeyOnlyFilter extends FilterBase { } } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof KeyOnlyFilter))) { + return false; + } + KeyOnlyFilter f = (KeyOnlyFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.lenAsVal); + } + static class KeyOnlyByteBufferExtendedCell extends ByteBufferExtendedCell { - public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE - + Bytes.SIZEOF_BOOLEAN; + public static final int FIXED_OVERHEAD = ClassSize.OBJECT + ClassSize.REFERENCE + Bytes.SIZEOF_BOOLEAN; private ByteBufferExtendedCell cell; private boolean lenAsVal; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java index dfd6297771..f54183b16b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultiRowRangeFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -512,5 +513,41 @@ public class MultiRowRangeFilter extends FilterBase { || Bytes.compareTo(startRow, stopRow) < 0 || (Bytes.compareTo(startRow, stopRow) == 0 && stopRowInclusive == true); } + @Override + public boolean equals(Object obj){ + if (obj == null || (!(obj instanceof RowRange))) { + return false; + } + if (this == obj) { + return true; + } + RowRange rr = (RowRange) obj; + return Bytes.equals(this.stopRow, rr.getStopRow()) + &&Bytes.equals(this.startRow,this.getStartRow()) + &&this.startRowInclusive==rr.isStartRowInclusive() + &&this.stopRowInclusive==rr.isStopRowInclusive(); + } + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.stopRow), + Bytes.hashCode(this.startRow), + this.startRowInclusive, + this.stopRowInclusive); + } + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof MultiRowRangeFilter))) { + return false; + } + MultiRowRangeFilter f = (MultiRowRangeFilter) obj; + return this.areSerializedFieldsEqual(f); } + + @Override + public int hashCode() { + return Objects.hash(this.rangeList.toArray()); + } + } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java index 47feea7f36..fabc33646b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/MultipleColumnPrefixFilter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; import java.util.TreeSet; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -39,24 +40,24 @@ import org.apache.hadoop.hbase.util.Bytes; */ @InterfaceAudience.Public public class MultipleColumnPrefixFilter extends FilterBase { - protected byte [] hint = null; - protected TreeSet sortedPrefixes = createTreeSet(); + protected byte[] hint = null; + protected TreeSet sortedPrefixes = createTreeSet(); private final static int MAX_LOG_PREFIXES = 5; - public MultipleColumnPrefixFilter(final byte [][] prefixes) { + 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"); + throw new IllegalArgumentException("prefixes must be distinct"); } } } - public byte [][] getPrefix() { + public byte[][] getPrefix() { int count = 0; - byte [][] temp = new byte [sortedPrefixes.size()][]; - for (byte [] prefixes : sortedPrefixes) { - temp [count++] = prefixes; + byte[][] temp = new byte[sortedPrefixes.size()][]; + for (byte[] prefixes : sortedPrefixes) { + temp[count++] = prefixes; } return temp; } @@ -83,17 +84,16 @@ public class MultipleColumnPrefixFilter extends FilterBase { } public ReturnCode filterColumn(Cell cell) { - byte [] qualifier = CellUtil.cloneQualifier(cell); - TreeSet lesserOrEqualPrefixes = - (TreeSet) sortedPrefixes.headSet(qualifier, true); + byte[] qualifier = CellUtil.cloneQualifier(cell); + TreeSet lesserOrEqualPrefixes = (TreeSet) sortedPrefixes.headSet(qualifier, true); if (lesserOrEqualPrefixes.size() != 0) { - byte [] largestPrefixSmallerThanQualifier = lesserOrEqualPrefixes.last(); - + byte[] largestPrefixSmallerThanQualifier = lesserOrEqualPrefixes.last(); + if (Bytes.startsWith(qualifier, largestPrefixSmallerThanQualifier)) { return ReturnCode.INCLUDE; } - + if (lesserOrEqualPrefixes.size() == sortedPrefixes.size()) { return ReturnCode.NEXT_ROW; } else { @@ -106,10 +106,10 @@ public class MultipleColumnPrefixFilter extends FilterBase { } } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - byte [][] prefixes = new byte [filterArguments.size()][]; - for (int i = 0 ; i < filterArguments.size(); i++) { - byte [] columnPrefix = ParseFilter.removeQuotesFromByteArray(filterArguments.get(i)); + public static Filter createFilterFromArguments(ArrayList 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); @@ -119,11 +119,11 @@ public class MultipleColumnPrefixFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.MultipleColumnPrefixFilter.Builder builder = - FilterProtos.MultipleColumnPrefixFilter.newBuilder(); - for (byte [] element : sortedPrefixes) { - if (element != null) builder.addSortedPrefixes(UnsafeByteOperations.unsafeWrap(element)); + public byte[] toByteArray() { + FilterProtos.MultipleColumnPrefixFilter.Builder builder = FilterProtos.MultipleColumnPrefixFilter.newBuilder(); + for (byte[] element : sortedPrefixes) { + if (element != null) + builder.addSortedPrefixes(UnsafeByteOperations.unsafeWrap(element)); } return builder.build().toByteArray(); } @@ -134,8 +134,7 @@ public class MultipleColumnPrefixFilter extends FilterBase { * @throws DeserializationException * @see #toByteArray */ - public static MultipleColumnPrefixFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static MultipleColumnPrefixFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.MultipleColumnPrefixFilter proto; try { proto = FilterProtos.MultipleColumnPrefixFilter.parseFrom(pbBytes); @@ -143,7 +142,7 @@ public class MultipleColumnPrefixFilter extends FilterBase { throw new DeserializationException(e); } int numPrefixes = proto.getSortedPrefixesCount(); - byte [][] prefixes = new byte[numPrefixes][]; + byte[][] prefixes = new byte[numPrefixes][]; for (int i = 0; i < numPrefixes; ++i) { prefixes[i] = proto.getSortedPrefixes(i).toByteArray(); } @@ -158,10 +157,12 @@ public class MultipleColumnPrefixFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof MultipleColumnPrefixFilter)) return false; + if (o == this) + return true; + if (!(o instanceof MultipleColumnPrefixFilter)) + return false; - MultipleColumnPrefixFilter other = (MultipleColumnPrefixFilter)o; + MultipleColumnPrefixFilter other = (MultipleColumnPrefixFilter) o; return this.sortedPrefixes.equals(other.sortedPrefixes); } @@ -170,18 +171,18 @@ public class MultipleColumnPrefixFilter extends FilterBase { return PrivateCellUtil.createFirstOnRowCol(cell, hint, 0, hint.length); } - public TreeSet createTreeSet() { + public TreeSet createTreeSet() { return new TreeSet<>(new Comparator() { - @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 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 @@ -204,7 +205,22 @@ public class MultipleColumnPrefixFilter extends FilterBase { } } - return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(), - count, this.sortedPrefixes.size(), prefixes.toString()); + return String.format("%s (%d/%d): [%s]", this.getClass().getSimpleName(), count, this.sortedPrefixes.size(), + prefixes.toString()); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof MultipleColumnPrefixFilter))) { + return false; + } + MultipleColumnPrefixFilter f = (MultipleColumnPrefixFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.sortedPrefixes); } + } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java index 30ddf24a78..06f777d629 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PageFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -150,4 +151,18 @@ public class PageFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.pageSize; } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof PageFilter))) { + return false; + } + PageFilter f = (PageFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.pageSize); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java index df9177bf8a..e243933a5b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/PrefixFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.ByteBufferExtendedCell; import org.apache.hadoop.hbase.Cell; @@ -156,4 +157,18 @@ public class PrefixFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof PrefixFilter))) { + return false; + } + PrefixFilter f = (PrefixFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.getPrefix())); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java index b38c0108df..83fc5c4487 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/QualifierFilter.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; - +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.yetus.audience.InterfaceAudience; @@ -55,8 +55,7 @@ public class QualifierFilter extends CompareFilter { * Use {@link #QualifierFilter(CompareOperator, ByteArrayComparable)} instead. */ @Deprecated - public QualifierFilter(final CompareOp op, - final ByteArrayComparable qualifierComparator) { + public QualifierFilter(final CompareOp op, final ByteArrayComparable qualifierComparator) { super(op, qualifierComparator); } @@ -65,8 +64,7 @@ public class QualifierFilter extends CompareFilter { * @param op the compare op for column qualifier matching * @param qualifierComparator the comparator for column qualifier matching */ - public QualifierFilter(final CompareOperator op, - final ByteArrayComparable qualifierComparator) { + public QualifierFilter(final CompareOperator op, final ByteArrayComparable qualifierComparator) { super(op, qualifierComparator); } @@ -87,10 +85,10 @@ public class QualifierFilter extends CompareFilter { return ReturnCode.INCLUDE; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { + public static Filter createFilterFromArguments(ArrayList filterArguments) { ArrayList arguments = CompareFilter.extractArguments(filterArguments); - CompareOperator compareOp = (CompareOperator)arguments.get(0); - ByteArrayComparable comparator = (ByteArrayComparable)arguments.get(1); + CompareOperator compareOp = (CompareOperator) arguments.get(0); + ByteArrayComparable comparator = (ByteArrayComparable) arguments.get(1); return new QualifierFilter(compareOp, comparator); } @@ -98,9 +96,8 @@ public class QualifierFilter extends CompareFilter { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.QualifierFilter.Builder builder = - FilterProtos.QualifierFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.QualifierFilter.Builder builder = FilterProtos.QualifierFilter.newBuilder(); builder.setCompareFilter(super.convert()); return builder.build().toByteArray(); } @@ -111,16 +108,14 @@ public class QualifierFilter extends CompareFilter { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static QualifierFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + 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 CompareOperator valueCompareOp = - CompareOperator.valueOf(proto.getCompareFilter().getCompareOp().name()); + final CompareOperator valueCompareOp = CompareOperator.valueOf(proto.getCompareFilter().getCompareOp().name()); ByteArrayComparable valueComparator = null; try { if (proto.getCompareFilter().hasComparator()) { @@ -129,7 +124,7 @@ public class QualifierFilter extends CompareFilter { } catch (IOException ioe) { throw new DeserializationException(ioe); } - return new QualifierFilter(valueCompareOp,valueComparator); + return new QualifierFilter(valueCompareOp, valueComparator); } /** @@ -138,9 +133,25 @@ public class QualifierFilter extends CompareFilter { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof QualifierFilter)) return false; + if (o == this) + return true; + if (!(o instanceof QualifierFilter)) + return false; return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof QualifierFilter))) { + return false; + } + QualifierFilter f = (QualifierFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getCompareOperator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java index 58624d27ff..dcee2f655d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RandomRowFilter.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; +import java.util.Objects; import java.util.Random; import org.apache.hadoop.hbase.Cell; @@ -117,9 +118,8 @@ public class RandomRowFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.RandomRowFilter.Builder builder = - FilterProtos.RandomRowFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.RandomRowFilter.Builder builder = FilterProtos.RandomRowFilter.newBuilder(); builder.setChance(this.chance); return builder.build().toByteArray(); } @@ -130,8 +130,7 @@ public class RandomRowFilter extends FilterBase { * @throws DeserializationException * @see #toByteArray */ - public static RandomRowFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static RandomRowFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.RandomRowFilter proto; try { proto = FilterProtos.RandomRowFilter.parseFrom(pbBytes); @@ -148,10 +147,26 @@ public class RandomRowFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof RandomRowFilter)) return false; + if (o == this) + return true; + if (!(o instanceof RandomRowFilter)) + return false; - RandomRowFilter other = (RandomRowFilter)o; + RandomRowFilter other = (RandomRowFilter) o; return this.getChance() == other.getChance(); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof RandomRowFilter))) { + return false; + } + RandomRowFilter f = (RandomRowFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getChance()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java index 8eba3ba2fb..435caab38b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/RowFilter.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; - +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.yetus.audience.InterfaceAudience; @@ -160,4 +160,18 @@ public class RowFilter extends CompareFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof RowFilter))) { + return false; + } + RowFilter f = (RowFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getCompareOperator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java index 566c408d01..7436ee6a35 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueExcludeFilter.java @@ -23,7 +23,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; - +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CompareOperator; @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; /** @@ -56,8 +57,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, byte[])} */ @Deprecated - public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, - CompareOp compareOp, byte[] value) { + public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, CompareOp compareOp, byte[] value) { super(family, qualifier, compareOp, value); } @@ -72,8 +72,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * @param op operator * @param value value to compare column values against */ - public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, - CompareOperator op, byte[] value) { + public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, CompareOperator op, byte[] value) { super(family, qualifier, op, value); } @@ -95,8 +94,8 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, ByteArrayComparable)} */ @Deprecated - public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, - CompareOp compareOp, ByteArrayComparable comparator) { + public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, CompareOp compareOp, + ByteArrayComparable comparator) { super(family, qualifier, compareOp, comparator); } @@ -115,12 +114,11 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * @param op operator * @param comparator Comparator to use. */ - public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, - CompareOperator op, ByteArrayComparable comparator) { + public SingleColumnValueExcludeFilter(byte[] family, byte[] qualifier, CompareOperator op, + ByteArrayComparable comparator) { super(family, qualifier, op, comparator); } - /** * Constructor for protobuf deserialization only. * @param family @@ -133,11 +131,9 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * {@link #SingleColumnValueExcludeFilter(byte[], byte[], CompareOperator, ByteArrayComparable, boolean, boolean)} */ @Deprecated - protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, - final CompareOp compareOp, ByteArrayComparable comparator, final boolean filterIfMissing, - final boolean latestVersionOnly) { - this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, - filterIfMissing, latestVersionOnly); + protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, final CompareOp compareOp, + ByteArrayComparable comparator, final boolean filterIfMissing, final boolean latestVersionOnly) { + this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, filterIfMissing, latestVersionOnly); } /** @@ -149,16 +145,15 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * @param filterIfMissing * @param latestVersionOnly */ - protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, - final CompareOperator op, ByteArrayComparable comparator, final boolean filterIfMissing, - final boolean latestVersionOnly) { + protected SingleColumnValueExcludeFilter(final byte[] family, final byte[] qualifier, final CompareOperator op, + ByteArrayComparable comparator, final boolean filterIfMissing, final boolean latestVersionOnly) { super(family, qualifier, op, comparator, filterIfMissing, latestVersionOnly); } // We cleaned result row in FilterRow to be consistent with scanning process. @Override public boolean hasFilterRow() { - return true; + return true; } // Here we remove from row all key values from testing column @@ -174,12 +169,11 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { } } - public static Filter createFilterFromArguments(ArrayList filterArguments) { - SingleColumnValueFilter tempFilter = (SingleColumnValueFilter) - SingleColumnValueFilter.createFilterFromArguments(filterArguments); - SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter ( - tempFilter.getFamily(), tempFilter.getQualifier(), - tempFilter.getOperator(), tempFilter.getComparator()); + public static Filter createFilterFromArguments(ArrayList filterArguments) { + SingleColumnValueFilter tempFilter = (SingleColumnValueFilter) SingleColumnValueFilter + .createFilterFromArguments(filterArguments); + SingleColumnValueExcludeFilter filter = new SingleColumnValueExcludeFilter(tempFilter.getFamily(), + tempFilter.getQualifier(), tempFilter.getOperator(), tempFilter.getComparator()); if (filterArguments.size() == 6) { filter.setFilterIfMissing(tempFilter.getFilterIfMissing()); @@ -192,9 +186,9 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { - FilterProtos.SingleColumnValueExcludeFilter.Builder builder = - FilterProtos.SingleColumnValueExcludeFilter.newBuilder(); + public byte[] toByteArray() { + FilterProtos.SingleColumnValueExcludeFilter.Builder builder = FilterProtos.SingleColumnValueExcludeFilter + .newBuilder(); builder.setSingleColumnValueFilter(super.convert()); return builder.build().toByteArray(); } @@ -205,8 +199,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { * @throws DeserializationException * @see #toByteArray */ - public static SingleColumnValueExcludeFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static SingleColumnValueExcludeFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.SingleColumnValueExcludeFilter proto; try { proto = FilterProtos.SingleColumnValueExcludeFilter.parseFrom(pbBytes); @@ -215,8 +208,7 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { } FilterProtos.SingleColumnValueFilter parentProto = proto.getSingleColumnValueFilter(); - final CompareOperator compareOp = - CompareOperator.valueOf(parentProto.getCompareOp().name()); + final CompareOperator compareOp = CompareOperator.valueOf(parentProto.getCompareOp().name()); final ByteArrayComparable comparator; try { comparator = ProtobufUtil.toComparator(parentProto.getComparator()); @@ -224,10 +216,10 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { throw new DeserializationException(ioe); } - return new SingleColumnValueExcludeFilter(parentProto.hasColumnFamily() ? parentProto - .getColumnFamily().toByteArray() : null, parentProto.hasColumnQualifier() ? parentProto - .getColumnQualifier().toByteArray() : null, compareOp, comparator, parentProto - .getFilterIfMissing(), parentProto.getLatestVersionOnly()); + return new SingleColumnValueExcludeFilter( + parentProto.hasColumnFamily() ? parentProto.getColumnFamily().toByteArray() : null, + parentProto.hasColumnQualifier() ? parentProto.getColumnQualifier().toByteArray() : null, compareOp, comparator, + parentProto.getFilterIfMissing(), parentProto.getLatestVersionOnly()); } /** @@ -236,9 +228,30 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof SingleColumnValueExcludeFilter)) return false; + if (o == this) + return true; + if (!(o instanceof SingleColumnValueExcludeFilter)) + return false; return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass().getName().equals(this.getClass().getName())))) { + return false; + } + SingleColumnValueExcludeFilter other = (SingleColumnValueExcludeFilter) obj; + return Bytes.equals(this.getFamily(), other.getFamily()) && Bytes.equals(this.getQualifier(), other.getQualifier()) + && this.op.equals(other.op) && this.getComparator().areSerializedFieldsEqual(other.getComparator()) + && this.getFilterIfMissing() == other.getFilterIfMissing() + && this.getLatestVersionOnly() == other.getLatestVersionOnly(); + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass().getName(), Bytes.hashCode(this.getFamily()), + Bytes.hashCode(this.getQualifier()), this.op, this.getComparator(), this.getFilterIfMissing(), + this.getLatestVersionOnly()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java index e5c83b1d72..d08f31b645 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java @@ -19,8 +19,9 @@ package org.apache.hadoop.hbase.filter; -import java.io.IOException; import java.util.ArrayList; +import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -70,8 +71,8 @@ import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; @InterfaceAudience.Public public class SingleColumnValueFilter extends FilterBase { - protected byte [] columnFamily; - protected byte [] columnQualifier; + protected byte[] columnFamily; + protected byte[] columnQualifier; protected CompareOperator op; protected org.apache.hadoop.hbase.filter.ByteArrayComparable comparator; protected boolean foundColumn = false; @@ -96,10 +97,10 @@ public class SingleColumnValueFilter extends FilterBase { * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, byte[])} instead. */ @Deprecated - public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, - final CompareOp compareOp, final byte[] value) { + public SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOp compareOp, + final byte[] value) { this(family, qualifier, CompareOperator.valueOf(compareOp.name()), - new org.apache.hadoop.hbase.filter.BinaryComparator(value)); + new org.apache.hadoop.hbase.filter.BinaryComparator(value)); } /** @@ -116,10 +117,9 @@ public class SingleColumnValueFilter extends FilterBase { * @param op operator * @param value value to compare column values against */ - public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, - final CompareOperator op, final byte[] value) { - this(family, qualifier, op, - new org.apache.hadoop.hbase.filter.BinaryComparator(value)); + public SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOperator op, + final byte[] value) { + this(family, qualifier, op, new org.apache.hadoop.hbase.filter.BinaryComparator(value)); } /** @@ -139,8 +139,7 @@ public class SingleColumnValueFilter extends FilterBase { * {@link #SingleColumnValueFilter(byte[], byte[], CompareOperator, ByteArrayComparable)} instead. */ @Deprecated - public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, - final CompareOp compareOp, + public SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOp compareOp, final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) { this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator); } @@ -159,8 +158,7 @@ public class SingleColumnValueFilter extends FilterBase { * @param op operator * @param comparator Comparator to use. */ - public SingleColumnValueFilter(final byte [] family, final byte [] qualifier, - final CompareOperator op, + public SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOperator op, final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator) { this.columnFamily = family; this.columnQualifier = qualifier; @@ -181,12 +179,10 @@ public class SingleColumnValueFilter extends FilterBase { * boolean, boolean)} instead. */ @Deprecated - protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, - final CompareOp compareOp, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, - final boolean filterIfMissing, + protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOp compareOp, + org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, final boolean filterIfMissing, final boolean latestVersionOnly) { - this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, filterIfMissing, - latestVersionOnly); + this(family, qualifier, CompareOperator.valueOf(compareOp.name()), comparator, filterIfMissing, latestVersionOnly); } /** @@ -198,9 +194,9 @@ public class SingleColumnValueFilter extends FilterBase { * @param filterIfMissing * @param latestVersionOnly */ - protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, - final CompareOperator op, org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, - final boolean filterIfMissing, final boolean latestVersionOnly) { + protected SingleColumnValueFilter(final byte[] family, final byte[] qualifier, final CompareOperator op, + org.apache.hadoop.hbase.filter.ByteArrayComparable comparator, final boolean filterIfMissing, + final boolean latestVersionOnly) { this(family, qualifier, op, comparator); this.filterIfMissing = filterIfMissing; this.latestVersionOnly = latestVersionOnly; @@ -267,7 +263,7 @@ public class SingleColumnValueFilter extends FilterBase { } foundColumn = true; if (filterColumnValue(c)) { - return this.latestVersionOnly? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE; + return this.latestVersionOnly ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE; } this.matchedColumn = true; return ReturnCode.INCLUDE; @@ -282,9 +278,9 @@ public class SingleColumnValueFilter extends FilterBase { public boolean filterRow() { // If column was found, return false if it was matched, true if it was not // If column not found, return true if we filter if missing, false if not - return this.foundColumn? !this.matchedColumn: this.filterIfMissing; + return this.foundColumn ? !this.matchedColumn : this.filterIfMissing; } - + @Override public boolean hasFilterRow() { return true; @@ -339,26 +335,23 @@ public class SingleColumnValueFilter extends FilterBase { this.latestVersionOnly = latestVersionOnly; } - public static Filter createFilterFromArguments(ArrayList filterArguments) { + public static Filter createFilterFromArguments(ArrayList filterArguments) { Preconditions.checkArgument(filterArguments.size() == 4 || filterArguments.size() == 6, - "Expected 4 or 6 but got: %s", filterArguments.size()); - byte [] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); - byte [] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)); + "Expected 4 or 6 but got: %s", filterArguments.size()); + byte[] family = ParseFilter.removeQuotesFromByteArray(filterArguments.get(0)); + byte[] qualifier = ParseFilter.removeQuotesFromByteArray(filterArguments.get(1)); CompareOperator op = ParseFilter.createCompareOperator(filterArguments.get(2)); - org.apache.hadoop.hbase.filter.ByteArrayComparable comparator = ParseFilter.createComparator( - ParseFilter.removeQuotesFromByteArray(filterArguments.get(3))); - - if (comparator instanceof RegexStringComparator || - comparator instanceof SubstringComparator) { - if (op != CompareOperator.EQUAL && - op != CompareOperator.NOT_EQUAL) { - throw new IllegalArgumentException ("A regexstring comparator and substring comparator " + - "can only be used with EQUAL and NOT_EQUAL"); + org.apache.hadoop.hbase.filter.ByteArrayComparable comparator = ParseFilter + .createComparator(ParseFilter.removeQuotesFromByteArray(filterArguments.get(3))); + + if (comparator instanceof RegexStringComparator || comparator instanceof SubstringComparator) { + if (op != CompareOperator.EQUAL && op != CompareOperator.NOT_EQUAL) { + throw new IllegalArgumentException( + "A regexstring comparator and substring comparator " + "can only be used with EQUAL and NOT_EQUAL"); } } - SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier, - op, comparator); + SingleColumnValueFilter filter = new SingleColumnValueFilter(family, qualifier, op, comparator); if (filterArguments.size() == 6) { boolean filterIfMissing = ParseFilter.convertByteArrayToBoolean(filterArguments.get(4)); @@ -370,8 +363,7 @@ public class SingleColumnValueFilter extends FilterBase { } FilterProtos.SingleColumnValueFilter convert() { - FilterProtos.SingleColumnValueFilter.Builder builder = - FilterProtos.SingleColumnValueFilter.newBuilder(); + FilterProtos.SingleColumnValueFilter.Builder builder = FilterProtos.SingleColumnValueFilter.newBuilder(); if (this.columnFamily != null) { builder.setColumnFamily(UnsafeByteOperations.unsafeWrap(this.columnFamily)); } @@ -391,7 +383,7 @@ public class SingleColumnValueFilter extends FilterBase { * @return The filter serialized using pb */ @Override - public byte [] toByteArray() { + public byte[] toByteArray() { return convert().toByteArray(); } @@ -401,8 +393,7 @@ public class SingleColumnValueFilter extends FilterBase { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static SingleColumnValueFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static SingleColumnValueFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.SingleColumnValueFilter proto; try { proto = FilterProtos.SingleColumnValueFilter.parseFrom(pbBytes); @@ -410,8 +401,7 @@ public class SingleColumnValueFilter extends FilterBase { throw new DeserializationException(e); } - final CompareOperator compareOp = - CompareOperator.valueOf(proto.getCompareOp().name()); + final CompareOperator compareOp = CompareOperator.valueOf(proto.getCompareOp().name()); final org.apache.hadoop.hbase.filter.ByteArrayComparable comparator; try { comparator = ProtobufUtil.toComparator(proto.getComparator()); @@ -419,10 +409,9 @@ public class SingleColumnValueFilter extends FilterBase { throw new DeserializationException(ioe); } - return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily() - .toByteArray() : null, proto.hasColumnQualifier() ? proto.getColumnQualifier() - .toByteArray() : null, compareOp, comparator, proto.getFilterIfMissing(), proto - .getLatestVersionOnly()); + return new SingleColumnValueFilter(proto.hasColumnFamily() ? proto.getColumnFamily().toByteArray() : null, + proto.hasColumnQualifier() ? proto.getColumnQualifier().toByteArray() : null, compareOp, comparator, + proto.getFilterIfMissing(), proto.getLatestVersionOnly()); } /** @@ -431,16 +420,16 @@ public class SingleColumnValueFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof SingleColumnValueFilter)) return false; - - SingleColumnValueFilter other = (SingleColumnValueFilter)o; - return Bytes.equals(this.getFamily(), other.getFamily()) - && Bytes.equals(this.getQualifier(), other.getQualifier()) - && this.op.equals(other.op) - && this.getComparator().areSerializedFieldsEqual(other.getComparator()) - && this.getFilterIfMissing() == other.getFilterIfMissing() - && this.getLatestVersionOnly() == other.getLatestVersionOnly(); + if (o == this) + return true; + if (!(o instanceof SingleColumnValueFilter)) + return false; + + SingleColumnValueFilter other = (SingleColumnValueFilter) o; + return Bytes.equals(this.getFamily(), other.getFamily()) && Bytes.equals(this.getQualifier(), other.getQualifier()) + && this.op.equals(other.op) && this.getComparator().areSerializedFieldsEqual(other.getComparator()) + && this.getFilterIfMissing() == other.getFilterIfMissing() + && this.getLatestVersionOnly() == other.getLatestVersionOnly(); } /** @@ -455,9 +444,23 @@ public class SingleColumnValueFilter extends FilterBase { @Override public String toString() { - return String.format("%s (%s, %s, %s, %s)", - this.getClass().getSimpleName(), Bytes.toStringBinary(this.columnFamily), - Bytes.toStringBinary(this.columnQualifier), this.op.name(), + return String.format("%s (%s, %s, %s, %s)", this.getClass().getSimpleName(), + Bytes.toStringBinary(this.columnFamily), Bytes.toStringBinary(this.columnQualifier), this.op.name(), Bytes.toStringBinary(this.comparator.getValue())); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass().getName().equals(this.getClass().getName())))) { + return false; + } + SingleColumnValueFilter f = (SingleColumnValueFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(this.getFamily()), Bytes.hashCode(this.getQualifier()), this.op, + this.getComparator(), this.getFilterIfMissing(), this.getLatestVersionOnly()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java index c710548bdb..f7646faa87 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SkipFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -100,7 +101,7 @@ public class SkipFilter extends FilterBase { public boolean filterRow() { return filterRow; } - + @Override public boolean hasFilterRow() { return true; @@ -111,8 +112,7 @@ public class SkipFilter extends FilterBase { */ @Override public byte[] toByteArray() throws IOException { - FilterProtos.SkipFilter.Builder builder = - FilterProtos.SkipFilter.newBuilder(); + FilterProtos.SkipFilter.Builder builder = FilterProtos.SkipFilter.newBuilder(); builder.setFilter(ProtobufUtil.toFilter(this.filter)); return builder.build().toByteArray(); } @@ -123,8 +123,7 @@ public class SkipFilter extends FilterBase { * @throws DeserializationException * @see #toByteArray */ - public static SkipFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static SkipFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.SkipFilter proto; try { proto = FilterProtos.SkipFilter.parseFrom(pbBytes); @@ -145,10 +144,12 @@ public class SkipFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof SkipFilter)) return false; + if (o == this) + return true; + if (!(o instanceof SkipFilter)) + return false; - SkipFilter other = (SkipFilter)o; + SkipFilter other = (SkipFilter) o; return getFilter().areSerializedFieldsEqual(other.getFilter()); } @@ -161,4 +162,18 @@ public class SkipFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.filter.toString(); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + SkipFilter f = (SkipFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.filter); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java index b3a8eae499..deecee9d70 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/TimestampsFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.TreeSet; import org.apache.hadoop.hbase.Cell; @@ -127,7 +128,6 @@ public class TimestampsFilter extends FilterBase { return canHint ? ReturnCode.SEEK_NEXT_USING_HINT : ReturnCode.SKIP; } - /** * Pick the next cell that the scanner should seek to. Since this can skip any number of cells * any of which can be a delete this can resurect old data. @@ -162,9 +162,9 @@ public class TimestampsFilter extends FilterBase { return PrivateCellUtil.createFirstOnRowColTS(currentCell, nextTimestamp); } - public static Filter createFilterFromArguments(ArrayList filterArguments) { + public static Filter createFilterFromArguments(ArrayList filterArguments) { ArrayList timestamps = new ArrayList<>(filterArguments.size()); - for (int i = 0; ibytes * @see #toByteArray */ - public static TimestampsFilter parseFrom(final byte[] pbBytes) - throws DeserializationException { + public static TimestampsFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.TimestampsFilter proto; try { proto = FilterProtos.TimestampsFilter.parseFrom(pbBytes); } catch (InvalidProtocolBufferException e) { throw new DeserializationException(e); } - return new TimestampsFilter(proto.getTimestampsList(), - proto.hasCanHint() && proto.getCanHint()); + return new TimestampsFilter(proto.getTimestampsList(), proto.hasCanHint() && proto.getCanHint()); } /** @@ -208,10 +205,12 @@ public class TimestampsFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof TimestampsFilter)) return false; + if (o == this) + return true; + if (!(o instanceof TimestampsFilter)) + return false; - TimestampsFilter other = (TimestampsFilter)o; + TimestampsFilter other = (TimestampsFilter) o; return this.getTimestamps().equals(other.getTimestamps()); } @@ -235,7 +234,21 @@ public class TimestampsFilter extends FilterBase { } } - return String.format("%s (%d/%d): [%s] canHint: [%b]", this.getClass().getSimpleName(), - count, this.timestamps.size(), tsList.toString(), canHint); + return String.format("%s (%d/%d): [%s] canHint: [%b]", this.getClass().getSimpleName(), count, + this.timestamps.size(), tsList.toString(), canHint); + } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof TimestampsFilter))) { + return false; + } + TimestampsFilter f = (TimestampsFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getTimestamps().toArray()); } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java index 3faa111d11..e4c8a84c31 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/ValueFilter.java @@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; - +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CompareOperator; import org.apache.yetus.audience.InterfaceAudience; @@ -139,4 +139,18 @@ public class ValueFilter extends CompareFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof ValueFilter))) { + return false; + } + ValueFilter f = (ValueFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getCompareOperator()); + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java index 6c1a47f215..56c8f4daa7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/WhileMatchFilter.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -72,7 +73,8 @@ public class WhileMatchFilter extends FilterBase { @Override public boolean filterRowKey(Cell cell) throws IOException { - if (filterAllRemaining()) return true; + if (filterAllRemaining()) + return true; boolean value = filter.filterRowKey(cell); changeFAR(value); return value; @@ -102,7 +104,7 @@ public class WhileMatchFilter extends FilterBase { changeFAR(filterRow); return filterRow; } - + @Override public boolean hasFilterRow() { return true; @@ -113,8 +115,7 @@ public class WhileMatchFilter extends FilterBase { */ @Override public byte[] toByteArray() throws IOException { - FilterProtos.WhileMatchFilter.Builder builder = - FilterProtos.WhileMatchFilter.newBuilder(); + FilterProtos.WhileMatchFilter.Builder builder = FilterProtos.WhileMatchFilter.newBuilder(); builder.setFilter(ProtobufUtil.toFilter(this.filter)); return builder.build().toByteArray(); } @@ -125,8 +126,7 @@ public class WhileMatchFilter extends FilterBase { * @throws org.apache.hadoop.hbase.exceptions.DeserializationException * @see #toByteArray */ - public static WhileMatchFilter parseFrom(final byte [] pbBytes) - throws DeserializationException { + public static WhileMatchFilter parseFrom(final byte[] pbBytes) throws DeserializationException { FilterProtos.WhileMatchFilter proto; try { proto = FilterProtos.WhileMatchFilter.parseFrom(pbBytes); @@ -147,10 +147,12 @@ public class WhileMatchFilter extends FilterBase { */ @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) return true; - if (!(o instanceof WhileMatchFilter)) return false; + if (o == this) + return true; + if (!(o instanceof WhileMatchFilter)) + return false; - WhileMatchFilter other = (WhileMatchFilter)o; + WhileMatchFilter other = (WhileMatchFilter) o; return getFilter().areSerializedFieldsEqual(other.getFilter()); } @@ -163,4 +165,18 @@ public class WhileMatchFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.filter.toString(); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + WhileMatchFilter f = (WhileMatchFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.filter); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java index 24f750af04..c95f7bcc90 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlFilter.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.security.access; import java.io.IOException; import java.util.Map; - +import java.util.Objects; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -173,4 +173,25 @@ class AccessControlFilter extends FilterBase { throw new UnsupportedOperationException( "Serialization not supported. Intended for server-side use only."); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + if(this == obj){ + return true; + } + AccessControlFilter f=(AccessControlFilter)obj; + return this.authManager.equals(f.authManager) + &&this.table.equals(f.table) + &&this.user.equals(f.user) + &&this.strategy.equals(f.strategy) + &&this.cfVsMaxVersions.equals(f.cfVsMaxVersions); + } + + @Override + public int hashCode() { + return Objects.hash(this.authManager,this.table,this.strategy,this.user,this.cfVsMaxVersions); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java index 4f00e7daa3..0ebaf09c46 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java @@ -1038,6 +1038,23 @@ public class VisibilityController implements MasterCoprocessor, RegionCoprocesso deleteCellVisTagsFormat); return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP; } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + if(this == obj){ + return true; + } + DeleteVersionVisibilityExpressionFilter f=(DeleteVersionVisibilityExpressionFilter)obj; + return this.deleteCellVisTags.equals(f.deleteCellVisTags) + &&this.deleteCellVisTagsFormat.equals(f.deleteCellVisTagsFormat); + } + + @Override + public int hashCode() { + return Objects.hash(this.deleteCellVisTags,this.deleteCellVisTagsFormat); + } } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java index 2bde9b59b1..d7eb6e5ac3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityLabelFilter.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hbase.security.visibility; import java.io.IOException; import java.util.Map; - +import java.util.Objects; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; @@ -90,4 +90,21 @@ class VisibilityLabelFilter extends FilterBase { this.curFamilyMaxVersions = 0; this.curQualMetVersions = 0; } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + if(this == obj){ + return true; + } + VisibilityLabelFilter f=(VisibilityLabelFilter)obj; + return this.expEvaluator.equals(f.expEvaluator) + &&this.cfVsMaxVersions.equals(f.cfVsMaxVersions); + } + + @Override + public int hashCode() { + return Objects.hash(this.expEvaluator,this.cfVsMaxVersions); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java index 5bede2ad11..5735a981c0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/ColumnCountOnRowFilter.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -55,4 +56,21 @@ public final class ColumnCountOnRowFilter extends FilterBase { public static ColumnCountOnRowFilter parseFrom(byte[] bytes) throws DeserializationException { return new ColumnCountOnRowFilter(Bytes.toInt(bytes)); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + if (this == obj) { + return true; + } + ColumnCountOnRowFilter f = (ColumnCountOnRowFilter) obj; + return this.limit == f.limit; + } + + @Override + public int hashCode() { + return Objects.hash(this.limit); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java index 40628db3e3..e2a70b23b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterAllFilter.java @@ -54,11 +54,14 @@ public class FilterAllFilter extends FilterBase { @Override boolean areSerializedFieldsEqual(Filter o) { - if (o == this) + if (o == this) { return true; - if (!(o instanceof FilterAllFilter)) + } + if (!(o instanceof FilterAllFilter)) { return false; + } return true; } + } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java index a3e3359c1d..c3d8a033a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilter.java @@ -21,10 +21,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.io.IOException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java index 5f9515ac4d..eeed6c2be8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java @@ -22,10 +22,10 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; - -import java.io.IOException; +import java.util.Objects; import java.util.ArrayList; import java.util.Arrays; +import java.io.IOException; import java.util.List; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; @@ -988,6 +988,22 @@ public class TestFilterList { public boolean getTransformed() { return this.transformed; } + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + if(this==obj){ + return true; + } + TransformFilter f=(TransformFilter)obj; + return this.targetRetCode.equals(f.targetRetCode); + } + + @Override + public int hashCode() { + return Objects.hash(this.targetRetCode); + } } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java index 815643d441..b53d40c9a6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -52,8 +53,7 @@ import org.junit.experimental.categories.Category; public class TestSwitchToStreamRead { @ClassRule - public static final HBaseClassTestRule CLASS_RULE = - HBaseClassTestRule.forClass(TestSwitchToStreamRead.class); + public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestSwitchToStreamRead.class); private static final HBaseTestingUtility UTIL = new HBaseTestingUtility(); @@ -76,19 +76,15 @@ public class TestSwitchToStreamRead { } VALUE_PREFIX = sb.append("-").toString(); REGION = UTIL.createLocalHRegion( - TableDescriptorBuilder.newBuilder(TABLE_NAME) - .setColumnFamily( - ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBlocksize(1024).build()) - .build(), - null, null); + TableDescriptorBuilder.newBuilder(TABLE_NAME) + .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBlocksize(1024).build()).build(), + null, null); for (int i = 0; i < 900; i++) { - REGION - .put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(VALUE_PREFIX + i))); + REGION.put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(VALUE_PREFIX + i))); } REGION.flush(true); for (int i = 900; i < 1000; i++) { - REGION - .put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(VALUE_PREFIX + i))); + REGION.put(new Put(Bytes.toBytes(i)).addColumn(FAMILY, QUAL, Bytes.toBytes(VALUE_PREFIX + i))); } } @@ -101,8 +97,7 @@ public class TestSwitchToStreamRead { @Test public void test() throws IOException { try (RegionScannerImpl scanner = REGION.getScanner(new Scan())) { - StoreScanner storeScanner = - (StoreScanner) (scanner).getStoreHeapForTesting().getCurrentForTesting(); + StoreScanner storeScanner = (StoreScanner) (scanner).getStoreHeapForTesting().getCurrentForTesting(); for (KeyValueScanner kvs : storeScanner.getAllScannersForTesting()) { if (kvs instanceof StoreFileScanner) { StoreFileScanner sfScanner = (StoreFileScanner) kvs; @@ -145,12 +140,24 @@ public class TestSwitchToStreamRead { public boolean filterRowKey(Cell cell) throws IOException { return Bytes.toInt(cell.getRowArray(), cell.getRowOffset()) != 999; } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj.getClass() == this.getClass()))) { + return false; + } + return this == obj; + } + + @Override + public int hashCode() { + return Objects.hash(MatchLastRowKeyFilter.class.getName()); + } } private void testFilter(Filter filter) throws IOException { try (RegionScannerImpl scanner = REGION.getScanner(new Scan().setFilter(filter))) { - StoreScanner storeScanner = - (StoreScanner) (scanner).getStoreHeapForTesting().getCurrentForTesting(); + StoreScanner storeScanner = (StoreScanner) (scanner).getStoreHeapForTesting().getCurrentForTesting(); for (KeyValueScanner kvs : storeScanner.getAllScannersForTesting()) { if (kvs instanceof StoreFileScanner) { StoreFileScanner sfScanner = (StoreFileScanner) kvs; @@ -160,8 +167,7 @@ public class TestSwitchToStreamRead { } List cells = new ArrayList<>(); // should return before finishing the scan as we want to switch from pread to stream - assertTrue(scanner.next(cells, - ScannerContext.newBuilder().setTimeLimit(LimitScope.BETWEEN_CELLS, -1).build())); + assertTrue(scanner.next(cells, ScannerContext.newBuilder().setTimeLimit(LimitScope.BETWEEN_CELLS, -1).build())); assertTrue(cells.isEmpty()); scanner.shipped(); @@ -172,8 +178,7 @@ public class TestSwitchToStreamRead { assertFalse(sfScanner.getReader().shared); } } - assertFalse(scanner.next(cells, - ScannerContext.newBuilder().setTimeLimit(LimitScope.BETWEEN_CELLS, -1).build())); + assertFalse(scanner.next(cells, ScannerContext.newBuilder().setTimeLimit(LimitScope.BETWEEN_CELLS, -1).build())); Result result = Result.create(cells); assertEquals(VALUE_PREFIX + 999, Bytes.toString(result.getValue(FAMILY, QUAL))); cells.clear(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java index 861b83e2fe..c571020ad3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/querymatcher/TestUserScanQueryMatcher.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; + import java.io.IOException; import java.util.ArrayList; import java.util.List; diff --git a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java index 694fb6ab8c..4bc6997a5b 100644 --- a/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java +++ b/hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java @@ -21,9 +21,11 @@ import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -47,14 +49,13 @@ import scala.collection.mutable.MutableList; * and avoid sending the data back to the client to be filtered. */ @InterfaceAudience.Private -public class SparkSQLPushDownFilter extends FilterBase{ +public class SparkSQLPushDownFilter extends FilterBase { protected static final Logger log = LoggerFactory.getLogger(SparkSQLPushDownFilter.class); //The following values are populated with protobuffer DynamicLogicExpression dynamicLogicExpression; byte[][] valueFromQueryArray; - HashMap> - currentCellToColumnIndexMap; + HashMap> currentCellToColumnIndexMap; //The following values are transient HashMap columnToCurrentRowValueMap = null; @@ -64,45 +65,38 @@ public class SparkSQLPushDownFilter extends FilterBase{ String encoderClassName; - public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, - byte[][] valueFromQueryArray, - HashMap> - currentCellToColumnIndexMap, String encoderClassName) { + public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, byte[][] valueFromQueryArray, + HashMap> currentCellToColumnIndexMap, + String encoderClassName) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; this.currentCellToColumnIndexMap = currentCellToColumnIndexMap; this.encoderClassName = encoderClassName; } - public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, - byte[][] valueFromQueryArray, - MutableList fields, String encoderClassName) { + public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, byte[][] valueFromQueryArray, + MutableList fields, String encoderClassName) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; this.encoderClassName = encoderClassName; //generate family qualifier to index mapping - this.currentCellToColumnIndexMap = - new HashMap<>(); + this.currentCellToColumnIndexMap = new HashMap<>(); for (int i = 0; i < fields.size(); i++) { Field field = fields.apply(i); byte[] cfBytes = field.cfBytes(); - ByteArrayComparable familyByteComparable = - new ByteArrayComparable(cfBytes, 0, cfBytes.length); + ByteArrayComparable familyByteComparable = new ByteArrayComparable(cfBytes, 0, cfBytes.length); - HashMap qualifierIndexMap = - currentCellToColumnIndexMap.get(familyByteComparable); + HashMap qualifierIndexMap = currentCellToColumnIndexMap.get(familyByteComparable); if (qualifierIndexMap == null) { qualifierIndexMap = new HashMap<>(); currentCellToColumnIndexMap.put(familyByteComparable, qualifierIndexMap); } byte[] qBytes = field.colBytes(); - ByteArrayComparable qualifierByteComparable = - new ByteArrayComparable(qBytes, 0, qBytes.length); + ByteArrayComparable qualifierByteComparable = new ByteArrayComparable(qBytes, 0, qBytes.length); qualifierIndexMap.put(qualifierByteComparable, field.colName()); } @@ -115,60 +109,46 @@ public class SparkSQLPushDownFilter extends FilterBase{ // the row key if (columnToCurrentRowValueMap == null) { columnToCurrentRowValueMap = new HashMap<>(); - HashMap qualifierColumnMap = - currentCellToColumnIndexMap.get( - new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); + HashMap qualifierColumnMap = currentCellToColumnIndexMap + .get(new ByteArrayComparable(rowKeyFamily, 0, rowKeyFamily.length)); if (qualifierColumnMap != null) { - String rowKeyColumnName = - qualifierColumnMap.get( - new ByteArrayComparable(rowKeyQualifier, 0, - rowKeyQualifier.length)); + String rowKeyColumnName = qualifierColumnMap + .get(new ByteArrayComparable(rowKeyQualifier, 0, rowKeyQualifier.length)); //Make sure that the rowKey is part of the where clause if (rowKeyColumnName != null) { columnToCurrentRowValueMap.put(rowKeyColumnName, - new ByteArrayComparable(c.getRowArray(), - c.getRowOffset(), c.getRowLength())); + new ByteArrayComparable(c.getRowArray(), c.getRowOffset(), c.getRowLength())); } } } //Always populate the column value into the RowValueMap - ByteArrayComparable currentFamilyByteComparable = - new ByteArrayComparable(c.getFamilyArray(), - c.getFamilyOffset(), - c.getFamilyLength()); + ByteArrayComparable currentFamilyByteComparable = new ByteArrayComparable(c.getFamilyArray(), c.getFamilyOffset(), + c.getFamilyLength()); - HashMap qualifierColumnMap = - currentCellToColumnIndexMap.get( - currentFamilyByteComparable); + HashMap qualifierColumnMap = currentCellToColumnIndexMap + .get(currentFamilyByteComparable); if (qualifierColumnMap != null) { - String columnName = - qualifierColumnMap.get( - new ByteArrayComparable(c.getQualifierArray(), - c.getQualifierOffset(), - c.getQualifierLength())); + String columnName = qualifierColumnMap + .get(new ByteArrayComparable(c.getQualifierArray(), c.getQualifierOffset(), c.getQualifierLength())); if (columnName != null) { columnToCurrentRowValueMap.put(columnName, - new ByteArrayComparable(c.getValueArray(), - c.getValueOffset(), c.getValueLength())); + new ByteArrayComparable(c.getValueArray(), c.getValueOffset(), c.getValueLength())); } } return ReturnCode.INCLUDE; } - @Override public boolean filterRow() throws IOException { try { - boolean result = - dynamicLogicExpression.execute(columnToCurrentRowValueMap, - valueFromQueryArray); + boolean result = dynamicLogicExpression.execute(columnToCurrentRowValueMap, valueFromQueryArray); columnToCurrentRowValueMap = null; return !result; } catch (Throwable e) { @@ -177,15 +157,13 @@ public class SparkSQLPushDownFilter extends FilterBase{ return false; } - /** * @param pbBytes A pb serialized instance * @return An instance of SparkSQLPushDownFilter * @throws DeserializationException if the filter cannot be parsed from the given bytes */ @SuppressWarnings("unused") - public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) - throws DeserializationException { + public static SparkSQLPushDownFilter parseFrom(final byte[] pbBytes) throws DeserializationException { SparkFilterProtos.SQLPredicatePushDownFilter proto; try { @@ -198,8 +176,8 @@ public class SparkSQLPushDownFilter extends FilterBase{ BytesEncoder enc = JavaBytesEncoder.create(encoder); //Load DynamicLogicExpression - DynamicLogicExpression dynamicLogicExpression = - DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc); + DynamicLogicExpression dynamicLogicExpression = DynamicLogicExpressionBuilder + .build(proto.getDynamicLogicExpression(), enc); //Load valuesFromQuery final List valueFromQueryArrayList = proto.getValueFromQueryArrayList(); @@ -209,36 +187,28 @@ public class SparkSQLPushDownFilter extends FilterBase{ } //Load mapping from HBase family/qualifier to Spark SQL columnName - HashMap> - currentCellToColumnIndexMap = new HashMap<>(); + HashMap> currentCellToColumnIndexMap = new HashMap<>(); - for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping - sqlPredicatePushDownCellToColumnMapping : - proto.getCellToColumnMappingList()) { + for (SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping sqlPredicatePushDownCellToColumnMapping : proto + .getCellToColumnMappingList()) { - byte[] familyArray = - sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); - ByteArrayComparable familyByteComparable = - new ByteArrayComparable(familyArray, 0, familyArray.length); - HashMap qualifierMap = - currentCellToColumnIndexMap.get(familyByteComparable); + byte[] familyArray = sqlPredicatePushDownCellToColumnMapping.getColumnFamily().toByteArray(); + ByteArrayComparable familyByteComparable = new ByteArrayComparable(familyArray, 0, familyArray.length); + HashMap qualifierMap = currentCellToColumnIndexMap.get(familyByteComparable); if (qualifierMap == null) { qualifierMap = new HashMap<>(); currentCellToColumnIndexMap.put(familyByteComparable, qualifierMap); } - byte[] qualifierArray = - sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); + byte[] qualifierArray = sqlPredicatePushDownCellToColumnMapping.getQualifier().toByteArray(); - ByteArrayComparable qualifierByteComparable = - new ByteArrayComparable(qualifierArray, 0 ,qualifierArray.length); + ByteArrayComparable qualifierByteComparable = new ByteArrayComparable(qualifierArray, 0, qualifierArray.length); - qualifierMap.put(qualifierByteComparable, - sqlPredicatePushDownCellToColumnMapping.getColumnName()); + qualifierMap.put(qualifierByteComparable, sqlPredicatePushDownCellToColumnMapping.getColumnName()); } - return new SparkSQLPushDownFilter(dynamicLogicExpression, - valueFromQueryArray, currentCellToColumnIndexMap, encoder); + return new SparkSQLPushDownFilter(dynamicLogicExpression, valueFromQueryArray, currentCellToColumnIndexMap, + encoder); } /** @@ -246,32 +216,58 @@ public class SparkSQLPushDownFilter extends FilterBase{ */ public byte[] toByteArray() { - SparkFilterProtos.SQLPredicatePushDownFilter.Builder builder = - SparkFilterProtos.SQLPredicatePushDownFilter.newBuilder(); + SparkFilterProtos.SQLPredicatePushDownFilter.Builder builder = SparkFilterProtos.SQLPredicatePushDownFilter + .newBuilder(); - SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder = - SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.newBuilder(); + SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping.Builder columnMappingBuilder = SparkFilterProtos.SQLPredicatePushDownCellToColumnMapping + .newBuilder(); builder.setDynamicLogicExpression(dynamicLogicExpression.toExpressionString()); - for (byte[] valueFromQuery: valueFromQueryArray) { + for (byte[] valueFromQuery : valueFromQueryArray) { builder.addValueFromQueryArray(ByteStringer.wrap(valueFromQuery)); } - for (Map.Entry> - familyEntry : currentCellToColumnIndexMap.entrySet()) { - for (Map.Entry qualifierEntry : - familyEntry.getValue().entrySet()) { - columnMappingBuilder.setColumnFamily( - ByteStringer.wrap(familyEntry.getKey().bytes())); - columnMappingBuilder.setQualifier( - ByteStringer.wrap(qualifierEntry.getKey().bytes())); + for (Map.Entry> familyEntry : currentCellToColumnIndexMap + .entrySet()) { + for (Map.Entry qualifierEntry : familyEntry.getValue().entrySet()) { + columnMappingBuilder.setColumnFamily(ByteStringer.wrap(familyEntry.getKey().bytes())); + columnMappingBuilder.setQualifier(ByteStringer.wrap(qualifierEntry.getKey().bytes())); columnMappingBuilder.setColumnName(qualifierEntry.getValue()); builder.addCellToColumnMapping(columnMappingBuilder.build()); } } builder.setEncoderClassName(encoderClassName); - return builder.build().toByteArray(); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof SparkSQLPushDownFilter))) { + return false; + } + if (this == obj) { + return true; + } + SparkSQLPushDownFilter f = (SparkSQLPushDownFilter) obj; + if (this.valueFromQueryArray.length != f.valueFromQueryArray.length) { + return false; + } + int i = 0; + for (byte[] val : this.valueFromQueryArray) { + if (!Bytes.equals(val, f.valueFromQueryArray[i])) { + return false; + } + i++; + } + return this.dynamicLogicExpression.toExpressionString().equals(f.dynamicLogicExpression.toExpressionString()) + && this.currentCellToColumnIndexMap.equals(f.currentCellToColumnIndexMap) + && this.encoderClassName.equals(f.encoderClassName); + } + + @Override + public int hashCode() { + return Objects.hash(this.dynamicLogicExpression, Arrays.hashCode(this.valueFromQueryArray), + this.currentCellToColumnIndexMap, this.encoderClassName); + } }