From 6f61938ecf7b1bd9ff4a85c32c9561f353996e1b Mon Sep 17 00:00:00 2001 From: liubangchen Date: Thu, 23 Aug 2018 11:19:17 +0800 Subject: [PATCH] HBASE-19008 Add missing equals or hashCode method(s) to stock Filter implementations Signed-off-by: Reid Chan Signed-off-by: Ted Yu and HBASE-21129 Clean up duplicate codes in #equals and #hashCode methods of Filter Signed-off-by Ted Yu --- .../hbase/filter/ColumnCountGetFilter.java | 11 ++++ .../hbase/filter/ColumnPaginationFilter.java | 12 +++++ .../hbase/filter/ColumnPrefixFilter.java | 10 ++++ .../hbase/filter/ColumnRangeFilter.java | 12 +++++ .../hadoop/hbase/filter/CompareFilter.java | 13 +++++ .../hbase/filter/DependentColumnFilter.java | 12 +++++ .../hadoop/hbase/filter/FamilyFilter.java | 10 ++++ .../hadoop/hbase/filter/FilterList.java | 11 ++++ .../hbase/filter/FilterListWithAND.java | 18 +++++++ .../hadoop/hbase/filter/FilterListWithOR.java | 21 ++++++++ .../hbase/filter/FirstKeyOnlyFilter.java | 11 ++++ ...FirstKeyValueMatchingQualifiersFilter.java | 15 +++++- .../hadoop/hbase/filter/FuzzyRowFilter.java | 11 ++++ .../hbase/filter/InclusiveStopFilter.java | 10 ++++ .../hadoop/hbase/filter/KeyOnlyFilter.java | 15 +++++- .../hbase/filter/MultiRowRangeFilter.java | 34 ++++++++++++ .../filter/MultipleColumnPrefixFilter.java | 26 +++++++--- .../hadoop/hbase/filter/PageFilter.java | 20 +++++-- .../hadoop/hbase/filter/PrefixFilter.java | 18 +++++-- .../hadoop/hbase/filter/QualifierFilter.java | 10 ++++ .../hadoop/hbase/filter/RandomRowFilter.java | 11 ++++ .../apache/hadoop/hbase/filter/RowFilter.java | 10 ++++ .../SingleColumnValueExcludeFilter.java | 10 ++++ .../hbase/filter/SingleColumnValueFilter.java | 12 +++++ .../hadoop/hbase/filter/SkipFilter.java | 11 ++++ .../hadoop/hbase/filter/TimestampsFilter.java | 11 ++++ .../hadoop/hbase/filter/ValueFilter.java | 19 +++++-- .../hadoop/hbase/filter/WhileMatchFilter.java | 11 ++++ .../security/access/AccessControlFilter.java | 23 ++++++++ .../visibility/VisibilityController.java | 19 +++++++ .../visibility/VisibilityLabelFilter.java | 19 +++++++ .../hbase/client/ColumnCountOnRowFilter.java | 20 ++++++- .../hadoop/hbase/filter/TestFilterList.java | 52 +++++++++++++++++++ 33 files changed, 504 insertions(+), 24 deletions(-) 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 d2f6ec84d4..d5e9997200 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 @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -127,4 +128,14 @@ public class ColumnCountGetFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.limit; } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 806863af07..e7303b11ee 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -236,4 +237,15 @@ public class ColumnPaginationFilter extends FilterBase return String.format("%s (%d, %d)", this.getClass().getSimpleName(), this.limit, this.offset); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return columnOffset == null ? Objects.hash(this.limit, this.offset) : + Objects.hash(this.limit, Bytes.hashCode(this.columnOffset)); + } } 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 eb0be8d62c..f295fe0bf8 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 @@ -149,4 +149,14 @@ public class ColumnPrefixFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return 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 22deca2419..0683063e88 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 @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.Bytes; import java.util.ArrayList; +import java.util.Objects; /** * This filter is used for selecting only those keys with columns that are @@ -242,4 +243,15 @@ public class ColumnRangeFilter extends FilterBase { + ", " + Bytes.toStringBinary(this.maxColumn) + (this.maxColumnInclusive ? "]" : ")"); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(getMinColumn()), getMinColumnInclusive(), + Bytes.hashCode(getMaxColumn()), getMaxColumnInclusive()); + } } 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 b7595d5641..fd8b10f159 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 com.google.common.base.Preconditions; import org.apache.hadoop.hbase.classification.InterfaceAudience; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; @@ -30,6 +31,8 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType; import org.apache.hadoop.hbase.util.Bytes; import java.util.ArrayList; +import java.util.Objects; + /** * This is a generic filter to be used to filter by comparison. It takes an * operator (equal, greater, not equal, etc) and a byte [] comparator. @@ -188,4 +191,14 @@ public abstract class CompareFilter extends FilterBase { this.compareOp.name(), Bytes.toStringBinary(this.comparator.getValue())); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(this.getComparator(), this.getOperator()); + } } 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 8582e38b1f..283e7dc8fc 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 @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Set; import org.apache.hadoop.hbase.util.ByteStringer; @@ -291,4 +292,15 @@ public class DependentColumnFilter extends CompareFilter { this.compareOp.name(), this.comparator != null ? Bytes.toStringBinary(this.comparator.getValue()) : "null"); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(getFamily()), Bytes.hashCode(getQualifier()), + dropDependentColumn(), getComparator(), getOperator()); + } } 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 9aa078c4bb..73ee6a9767 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 @@ -130,4 +130,14 @@ public class FamilyFilter extends CompareFilter { FamilyFilter other = (FamilyFilter)o; return super.areSerializedFieldsEqual(other); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } 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 5002957a30..11780859ea 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 com.google.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.hbase.Cell; @@ -295,4 +296,14 @@ final public class FilterList extends Filter { public String toString() { return this.filterListBase.toString(); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(getOperator(), getFilters()); + } } 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 a9a5f693a8..2e7255de18 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 @@ -260,4 +261,21 @@ public class FilterListWithAND extends FilterListBase { } return maxHint; } + + @Override + public boolean equals(Object obj) { + if (!(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); + } } 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 5293556503..4b3fd6b315 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 @@ -27,6 +27,7 @@ import java.io.IOException; import java.util.ArrayList; 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 @@ -377,4 +378,24 @@ 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(this.prevFilterRCList, this.prevCellList, this.filters); + } } 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 d18a1f8aaf..f1c0d0d591 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.filter; import java.util.ArrayList; +import java.util.Objects; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -122,4 +123,14 @@ public class FirstKeyOnlyFilter extends FilterBase { return true; } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hashCode(foundKV); + } } 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 2f1c037f25..9f8030ab27 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,10 +18,10 @@ package org.apache.hadoop.hbase.filter; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.ByteStringer; @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.util.Bytes; import com.google.protobuf.ByteString; import com.google.protobuf.InvalidProtocolBufferException; +import java.util.Objects; import java.util.Set; import java.util.TreeSet; @@ -127,4 +128,14 @@ public class FirstKeyValueMatchingQualifiersFilter extends FirstKeyOnlyFilter { FirstKeyValueMatchingQualifiersFilter other = (FirstKeyValueMatchingQualifiersFilter)o; return this.qualifiers.equals(other.qualifiers); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 a2bcfb812a..818ef0e9a2 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 @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.List; +import java.util.Objects; import java.util.PriorityQueue; import org.apache.hadoop.hbase.Cell; @@ -640,4 +641,14 @@ public class FuzzyRowFilter extends FilterBase { } return true; } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(this.fuzzyKeysData); + } } 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 671e596f05..c39c41713e 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 @@ -140,4 +140,14 @@ public class InclusiveStopFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.stopRowKey); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return 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 3895b2a82a..271400f560 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 @@ -21,12 +21,13 @@ package org.apache.hadoop.hbase.filter; import java.io.IOException; import java.util.ArrayList; +import java.util.Objects; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.Bytes; @@ -126,4 +127,14 @@ public class KeyOnlyFilter extends FilterBase { KeyOnlyFilter other = (KeyOnlyFilter)o; return this.lenAsVal == other.lenAsVal; } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(this.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 fd21dbd465..f5715b9abb 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 @@ -21,6 +21,7 @@ import java.io.IOException; 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.HConstants; @@ -512,5 +513,38 @@ 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 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) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(this.rangeList); } } 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 96a0b389a2..cd1b24b62d 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 @@ -18,20 +18,22 @@ package org.apache.hadoop.hbase.filter; import com.google.protobuf.InvalidProtocolBufferException; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.hadoop.hbase.classification.InterfaceStability; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Objects; +import java.util.TreeSet; + import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.TreeSet; - /** * This filter is used for selecting only those keys with columns that matches * a particular prefix. For example, if prefix is 'an', it will pass keys will @@ -206,4 +208,14 @@ 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) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 9692f2287e..7449d17a34 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,15 +20,17 @@ package org.apache.hadoop.hbase.filter; import com.google.common.base.Preconditions; import com.google.protobuf.InvalidProtocolBufferException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Objects; + +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; -import java.io.IOException; -import java.util.ArrayList; - /** * Implementation of Filter interface that limits results to a specific page * size. It terminates scanning once the number of filter-passed rows is > @@ -141,4 +143,14 @@ public class PageFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.pageSize; } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 a3fc440023..cb68a70a95 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,17 +20,17 @@ package org.apache.hadoop.hbase.filter; import com.google.common.base.Preconditions; -import org.apache.hadoop.hbase.util.ByteStringer; import com.google.protobuf.InvalidProtocolBufferException; +import java.util.ArrayList; + +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.Bytes; - -import java.util.ArrayList; +import org.apache.hadoop.hbase.util.ByteStringer; /** * Pass results that have same row prefix. @@ -149,4 +149,14 @@ public class PrefixFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + Bytes.toStringBinary(this.prefix); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return 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 1f7e8430aa..c433fedd7f 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 @@ -125,4 +125,14 @@ public class QualifierFilter extends CompareFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } 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 f6d091ffc5..7bb7a4641b 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.classification.InterfaceAudience; @@ -157,4 +158,14 @@ public class RandomRowFilter extends FilterBase { RandomRowFilter other = (RandomRowFilter)o; return this.getChance() == other.getChance(); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 ee32a927c1..a9d5b8aae0 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 @@ -144,4 +144,14 @@ public class RowFilter extends CompareFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } 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 7dc0387046..01f8160864 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 @@ -184,4 +184,14 @@ public class SingleColumnValueExcludeFilter extends SingleColumnValueFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } 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 0cf3e2f47c..2c6b919092 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 @@ -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.util.ByteStringer; import org.apache.commons.logging.Log; @@ -399,4 +400,15 @@ public class SingleColumnValueFilter extends FilterBase { Bytes.toStringBinary(this.columnQualifier), this.compareOp.name(), Bytes.toStringBinary(this.comparator.getValue())); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(Bytes.hashCode(getFamily()), Bytes.hashCode(getQualifier()), + this.getOperator(), getComparator(), getFilterIfMissing(), 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 9bc18a45b4..d7bb550556 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -152,4 +153,14 @@ public class SkipFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.filter.toString(); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 f28560bfd6..1df969c7e6 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 @@ -22,6 +22,7 @@ import com.google.protobuf.InvalidProtocolBufferException; 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; @@ -233,4 +234,14 @@ 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) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return Objects.hash(getTimestamps()); + } } 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 952d64ec86..bd934571b4 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 @@ -20,16 +20,17 @@ package org.apache.hadoop.hbase.filter; import com.google.protobuf.InvalidProtocolBufferException; + +import java.io.IOException; +import java.util.ArrayList; + +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; -import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.FilterProtos; -import java.io.IOException; -import java.util.ArrayList; - /** * This filter is used to filter based on column value. It takes an * operator (equal, greater, not equal, etc) and a byte [] comparator for the @@ -124,4 +125,14 @@ public class ValueFilter extends CompareFilter { return super.areSerializedFieldsEqual(o); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @Override + public int hashCode() { + return super.hashCode(); + } } 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 1cefe46e90..0ff3719434 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.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; @@ -152,4 +153,14 @@ public class WhileMatchFilter extends FilterBase { public String toString() { return this.getClass().getSimpleName() + " " + this.filter.toString(); } + + @Override + public boolean equals(Object obj) { + return obj instanceof Filter && areSerializedFieldsEqual((Filter) obj); + } + + @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 40c12b78f5..9c564ee13a 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,6 +20,7 @@ package org.apache.hadoop.hbase.security.access; import java.io.IOException; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; @@ -174,4 +175,26 @@ class AccessControlFilter extends FilterBase { throw new UnsupportedOperationException( "Serialization not supported. Intended for server-side use only."); } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof AccessControlFilter)) { + 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 0883354c87..a1569d84ef 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 @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -1117,6 +1118,24 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements public Cell transformCell(Cell v) { return v; } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DeleteVersionVisibilityExpressionFilter)) { + 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 18fc4668ba..57d27636fc 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,6 +19,7 @@ package org.apache.hadoop.hbase.security.visibility; import java.io.IOException; import java.util.Map; +import java.util.Objects; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Cell; @@ -92,4 +93,22 @@ class VisibilityLabelFilter extends FilterBase { this.curFamilyMaxVersions = 0; this.curQualMetVersions = 0; } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof VisibilityLabelFilter)) { + 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 c4b4d2879b..1de561a323 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.hadoop.hbase.classification.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)); } -} \ No newline at end of file + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof ColumnCountOnRowFilter)) { + 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/TestFilterList.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/TestFilterList.java index 918c94c26a..2d44c817c9 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 @@ -30,6 +30,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Objects; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; @@ -635,6 +636,23 @@ public class TestFilterList { this.didCellPassToTheFilter = true; return targetRetCode; } + + @Override + public boolean equals(Object obj) { + if(obj == null || !(obj instanceof MockFilter)){ + return false; + } + if(obj == this){ + return true; + } + MockFilter f = (MockFilter)obj; + return this.targetRetCode.equals(f.targetRetCode); + } + + @Override + public int hashCode() { + return Objects.hash(this.targetRetCode); + } } @Test @@ -807,6 +825,23 @@ public class TestFilterList { public Cell getNextCellHint(Cell currentCell) throws IOException { return this.returnCell; } + + @Override + public boolean equals(Object obj) { + if(obj == null || !(obj instanceof MockSeekHintFilter)){ + return false; + } + if(obj == this){ + return true; + } + MockSeekHintFilter f = (MockSeekHintFilter)obj; + return this.returnCell.equals(f.returnCell); + } + + @Override + public int hashCode() { + return Objects.hash(this.returnCell); + } } @Test @@ -975,6 +1010,23 @@ public class TestFilterList { public boolean getTransformed() { return this.transformed; } + + @Override + public boolean equals(Object obj) { + if(!(obj instanceof TransformFilter)){ + return false; + } + if (obj == this) { + return true; + } + TransformFilter f = (TransformFilter)obj; + return this.targetRetCode.equals(f.targetRetCode); + } + + @Override + public int hashCode() { + return Objects.hash(this.targetRetCode); + } } @Test -- 2.18.0