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..e719305717 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,8 @@ 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 +81,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 +91,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 +103,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 +120,12 @@ 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 +133,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..81d35b8ac7 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; +import java.util.Objects; /** * This filter is used for selecting only those keys with columns that matches @@ -161,4 +162,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(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..3c36c2288d 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,31 @@ 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 +192,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 +231,25 @@ 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..703d1c92c4 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(this.getFamily(), 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..0b1efe2cd9 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(this.getFamily(),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..8c3831d59a 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(), 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..a029f0ede2 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 @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.ArrayList; 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 +280,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, 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..03d36f92c9 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 @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.util.Objects; import java.io.IOException; import java.util.ArrayList; 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..389cd7e6a8 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 @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.util.UnsafeAccess; import org.apache.hadoop.hbase.util.UnsafeAvailChecker; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; - +import java.util.Objects; /** * This is optimized version of a standard FuzzyRowFilter Filters data based on fuzzy row key. * Performs fast-forwards during scanning. It takes pairs (row key, fuzzy info) to match row keys. @@ -100,7 +100,6 @@ public class FuzzyRowFilter extends FilterBase { this.tracker = new RowTracker(); } - private void preprocessSearchKey(Pair p) { if (!UNSAFE_UNALIGNED) { // do nothing @@ -646,4 +645,18 @@ public class FuzzyRowFilter extends FilterBase { } 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..faab8248ab 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(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..77ebd73a86 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 @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; import org.apache.yetus.audience.InterfaceAudience; +import java.util.Objects; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException; @@ -56,7 +57,9 @@ public class KeyOnlyFilter extends FilterBase { // Impl in FilterBase might do unnecessary copy for Off heap backed Cells. return false; } - + public boolean isLenAsVal(){ + return this.lenAsVal; + } @Override public Cell transformCell(Cell cell) { return createKeyOnlyCell(cell); @@ -246,7 +249,19 @@ public class KeyOnlyFilter extends FilterBase { return 0; } } - + @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; 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..6366d4d633 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 @@ -31,6 +31,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferExce import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.Bytes; +import java.util.Objects; /** * This filter is used for selecting only those keys with columns that matches @@ -207,4 +208,19 @@ public class MultipleColumnPrefixFilter extends FilterBase { 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..fa504c9b41 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(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..4769996879 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; @@ -143,4 +143,18 @@ public class QualifierFilter extends CompareFilter { 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..5e718e1679 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.Random; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; @@ -154,4 +155,18 @@ public class RandomRowFilter extends FilterBase { 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..4f1bd6723f 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; @@ -241,4 +241,20 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof SingleColumnValueExcludeFilter))) { + return false; + } + SingleColumnValueExcludeFilter f = (SingleColumnValueExcludeFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass().getName(), this.getFamily(), 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..e446f044cc 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 @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.CompareType; import org.apache.hadoop.hbase.util.Bytes; - +import java.util.Objects; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; /** @@ -460,4 +460,20 @@ public class SingleColumnValueFilter extends FilterBase { Bytes.toStringBinary(this.columnQualifier), this.op.name(), Bytes.toStringBinary(this.comparator.getValue())); } + + @Override + public boolean equals(Object obj) { + if (obj == null || (!(obj instanceof SingleColumnValueFilter))) { + return false; + } + SingleColumnValueFilter f = (SingleColumnValueFilter) obj; + return this.areSerializedFieldsEqual(f); + } + + @Override + public int hashCode() { + return Objects.hash(this.getFamily(), 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..58a37f007a 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; @@ -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..3d05c0d168 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 @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.TreeSet; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.PrivateCellUtil; @@ -238,4 +239,18 @@ public class TimestampsFilter extends FilterBase { 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..68e81ab50c 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; @@ -163,4 +164,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/regionserver/MobReferenceOnlyFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java index b544ea443d..fc375cbac3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobReferenceOnlyFilter.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.filter.FilterBase; 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/AbstractTestScanCursor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java index 14f2caa326..97fa5c3032 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestScanCursor.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; 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..a8c5c76eab 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/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index bdea5fd2a8..614c0436f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -73,6 +73,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import java.util.Objects; @Category({CoprocessorTests.class, MediumTests.class}) public class TestRegionObserverScannerOpenHook { 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..9a10dc6aea 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 @@ -21,6 +21,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; +import java.util.Objects; public class FilterAllFilter extends FilterBase { @@ -61,4 +62,12 @@ public class FilterAllFilter extends FilterBase { return true; } + @Override + public boolean equals(Object obj) { + if (!(obj instanceof FilterAllFilter)) + return false; + FilterAllFilter f=(FilterAllFilter)obj; + return this.areSerializedFieldsEqual(f); + } + } 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..59db7e85ab 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.Objects; import java.io.IOException; import java.util.ArrayList; 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..d0037935df 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,7 +22,7 @@ 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.util.Objects; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -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/filter/TestInvocationRecordFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java index 5184b14f75..ba07e07f84 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestInvocationRecordFilter.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.filter; +import java.util.Objects; import java.util.ArrayList; import java.util.List; import org.apache.hadoop.hbase.Cell; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java index 5aec32aa3d..3a47872778 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerHeartbeatMessages.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Objects; import java.io.IOException; import java.util.ArrayList; 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..cc8a0a649b 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.Objects; import java.io.IOException; import java.util.ArrayList; @@ -145,6 +146,18 @@ 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 { 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..af3c54a364 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 @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver.querymatcher; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import java.util.Objects; + import java.io.IOException; import java.util.ArrayList; 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..2bd2cc0038 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; @@ -274,4 +276,34 @@ public class SparkSQLPushDownFilter extends FilterBase{ 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); + } }