mirror of
https://github.com/apache/druid.git
synced 2025-02-23 11:06:19 +00:00
Cleaned up equals() and hashCode(), per code review
This commit is contained in:
parent
4b63645802
commit
be1ef3a161
@ -132,35 +132,6 @@ public abstract class BaseQuery<T> implements Query<T>
|
|||||||
return overridden;
|
return overridden;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Compare the BaseQuery fields with another BaseQuery. For use in subclasses implementing equals()
|
|
||||||
* @param that
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
protected boolean partialEquals(BaseQuery that)
|
|
||||||
{
|
|
||||||
if (context != null ? !context.equals(that.context) : that.context != null) return false;
|
|
||||||
if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) return false;
|
|
||||||
if (duration != null ? !duration.equals(that.duration) : that.duration != null) return false;
|
|
||||||
if (querySegmentSpec != null ? !querySegmentSpec.equals(that.querySegmentSpec) : that.querySegmentSpec != null)
|
|
||||||
return false;
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Hash the fields within BaseQuery. For use in subclasses implementing hashCode()
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
protected int partialHashCode()
|
|
||||||
{
|
|
||||||
int result = dataSource != null ? dataSource.hashCode() : 0;
|
|
||||||
result = 31 * result + (context != null ? context.hashCode() : 0);
|
|
||||||
result = 31 * result + (querySegmentSpec != null ? querySegmentSpec.hashCode() : 0);
|
|
||||||
result = 31 * result + (duration != null ? duration.hashCode() : 0);
|
|
||||||
return result;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getId()
|
public String getId()
|
||||||
{
|
{
|
||||||
@ -172,4 +143,31 @@ public abstract class BaseQuery<T> implements Query<T>
|
|||||||
{
|
{
|
||||||
return withOverriddenContext(ImmutableMap.of(QUERYID, id));
|
return withOverriddenContext(ImmutableMap.of(QUERYID, id));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
BaseQuery baseQuery = (BaseQuery) o;
|
||||||
|
|
||||||
|
if (context != null ? !context.equals(baseQuery.context) : baseQuery.context != null) return false;
|
||||||
|
if (dataSource != null ? !dataSource.equals(baseQuery.dataSource) : baseQuery.dataSource != null) return false;
|
||||||
|
if (duration != null ? !duration.equals(baseQuery.duration) : baseQuery.duration != null) return false;
|
||||||
|
if (querySegmentSpec != null ? !querySegmentSpec.equals(baseQuery.querySegmentSpec) : baseQuery.querySegmentSpec != null)
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = dataSource != null ? dataSource.hashCode() : 0;
|
||||||
|
result = 31 * result + (context != null ? context.hashCode() : 0);
|
||||||
|
result = 31 * result + (querySegmentSpec != null ? querySegmentSpec.hashCode() : 0);
|
||||||
|
result = 31 * result + (duration != null ? duration.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -24,7 +24,6 @@ package io.druid.query;
|
|||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
|
|
||||||
import java.io.Serializable;
|
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME,
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME,
|
||||||
include = JsonTypeInfo.As.PROPERTY,
|
include = JsonTypeInfo.As.PROPERTY,
|
||||||
property = "type",
|
property = "type",
|
||||||
@ -33,6 +32,6 @@ import java.io.Serializable;
|
|||||||
@JsonSubTypes.Type(value = TableDataSource.class, name = "table"),
|
@JsonSubTypes.Type(value = TableDataSource.class, name = "table"),
|
||||||
@JsonSubTypes.Type(value = QueryDataSource.class, name = "query")
|
@JsonSubTypes.Type(value = QueryDataSource.class, name = "query")
|
||||||
})
|
})
|
||||||
public interface DataSource extends Serializable
|
public interface DataSource
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
@ -507,10 +507,9 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||||||
{
|
{
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
GroupByQuery that = (GroupByQuery) o;
|
GroupByQuery that = (GroupByQuery) o;
|
||||||
if (!partialEquals(that))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null)
|
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null)
|
||||||
return false;
|
return false;
|
||||||
@ -519,6 +518,8 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||||||
if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false;
|
if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false;
|
||||||
if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false;
|
if (havingSpec != null ? !havingSpec.equals(that.havingSpec) : that.havingSpec != null) return false;
|
||||||
if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false;
|
if (limitSpec != null ? !limitSpec.equals(that.limitSpec) : that.limitSpec != null) return false;
|
||||||
|
if (orderByLimitFn != null ? !orderByLimitFn.equals(that.orderByLimitFn) : that.orderByLimitFn != null)
|
||||||
|
return false;
|
||||||
if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null)
|
if (postAggregatorSpecs != null ? !postAggregatorSpecs.equals(that.postAggregatorSpecs) : that.postAggregatorSpecs != null)
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -528,8 +529,8 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = limitSpec != null ? limitSpec.hashCode() : 0;
|
int result = super.hashCode();
|
||||||
result = 31 * result + partialHashCode();
|
result = 31 * result + (limitSpec != null ? limitSpec.hashCode() : 0);
|
||||||
result = 31 * result + (havingSpec != null ? havingSpec.hashCode() : 0);
|
result = 31 * result + (havingSpec != null ? havingSpec.hashCode() : 0);
|
||||||
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
||||||
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
||||||
|
@ -196,6 +196,25 @@ public class DefaultLimitSpec implements LimitSpec
|
|||||||
{
|
{
|
||||||
return Sequences.limit(input, limit);
|
return Sequences.limit(input, limit);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
LimitingFn that = (LimitingFn) o;
|
||||||
|
|
||||||
|
if (limit != that.limit) return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return limit;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class SortingFn implements Function<Sequence<Row>, Sequence<Row>>
|
private static class SortingFn implements Function<Sequence<Row>, Sequence<Row>>
|
||||||
@ -209,6 +228,25 @@ public class DefaultLimitSpec implements LimitSpec
|
|||||||
{
|
{
|
||||||
return Sequences.sort(input, ordering);
|
return Sequences.sort(input, ordering);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
SortingFn sortingFn = (SortingFn) o;
|
||||||
|
|
||||||
|
if (ordering != null ? !ordering.equals(sortingFn.ordering) : sortingFn.ordering != null) return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return ordering != null ? ordering.hashCode() : 0;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class TopNFunction implements Function<Sequence<Row>, Sequence<Row>>
|
private static class TopNFunction implements Function<Sequence<Row>, Sequence<Row>>
|
||||||
@ -231,6 +269,28 @@ public class DefaultLimitSpec implements LimitSpec
|
|||||||
final ArrayList<Row> materializedList = Sequences.toList(input, Lists.<Row>newArrayList());
|
final ArrayList<Row> materializedList = Sequences.toList(input, Lists.<Row>newArrayList());
|
||||||
return Sequences.simple(sorter.toTopN(materializedList, limit));
|
return Sequences.simple(sorter.toTopN(materializedList, limit));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
TopNFunction that = (TopNFunction) o;
|
||||||
|
|
||||||
|
if (limit != that.limit) return false;
|
||||||
|
if (sorter != null ? !sorter.equals(that.sorter) : that.sorter != null) return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = sorter != null ? sorter.hashCode() : 0;
|
||||||
|
result = 31 * result + limit;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -95,10 +95,9 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||||||
{
|
{
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
SegmentMetadataQuery that = (SegmentMetadataQuery) o;
|
SegmentMetadataQuery that = (SegmentMetadataQuery) o;
|
||||||
if (!partialEquals(that))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (merge != that.merge) return false;
|
if (merge != that.merge) return false;
|
||||||
if (toInclude != null ? !toInclude.equals(that.toInclude) : that.toInclude != null) return false;
|
if (toInclude != null ? !toInclude.equals(that.toInclude) : that.toInclude != null) return false;
|
||||||
@ -109,8 +108,8 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
|
|||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = toInclude != null ? toInclude.hashCode() : 0;
|
int result = super.hashCode();
|
||||||
result = 31 * result + partialHashCode();
|
result = 31 * result + (toInclude != null ? toInclude.hashCode() : 0);
|
||||||
result = 31 * result + (merge ? 1 : 0);
|
result = 31 * result + (merge ? 1 : 0);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
@ -194,12 +194,10 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
|
|||||||
{
|
{
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
SearchQuery that = (SearchQuery) o;
|
SearchQuery that = (SearchQuery) o;
|
||||||
|
|
||||||
if (!partialEquals(that))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (limit != that.limit) return false;
|
if (limit != that.limit) return false;
|
||||||
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false;
|
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false;
|
||||||
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false;
|
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false;
|
||||||
@ -213,8 +211,8 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
|
|||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = dimFilter != null ? dimFilter.hashCode() : 0;
|
int result = super.hashCode();
|
||||||
result = 31 * result + partialHashCode();
|
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
||||||
result = 31 * result + (sortSpec != null ? sortSpec.hashCode() : 0);
|
result = 31 * result + (sortSpec != null ? sortSpec.hashCode() : 0);
|
||||||
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
||||||
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
|
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
|
||||||
|
@ -147,4 +147,34 @@ public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
|
|||||||
", pagingSpec=" + pagingSpec +
|
", pagingSpec=" + pagingSpec +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o)
|
||||||
|
{
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
|
SelectQuery that = (SelectQuery) o;
|
||||||
|
|
||||||
|
if (dimFilter != null ? !dimFilter.equals(that.dimFilter) : that.dimFilter != null) return false;
|
||||||
|
if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) return false;
|
||||||
|
if (granularity != null ? !granularity.equals(that.granularity) : that.granularity != null) return false;
|
||||||
|
if (metrics != null ? !metrics.equals(that.metrics) : that.metrics != null) return false;
|
||||||
|
if (pagingSpec != null ? !pagingSpec.equals(that.pagingSpec) : that.pagingSpec != null) return false;
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
int result = super.hashCode();
|
||||||
|
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
||||||
|
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
||||||
|
result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0);
|
||||||
|
result = 31 * result + (metrics != null ? metrics.hashCode() : 0);
|
||||||
|
result = 31 * result + (pagingSpec != null ? pagingSpec.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -162,17 +162,4 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
|
|||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) return true;
|
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
|
||||||
|
|
||||||
return partialEquals((TimeBoundaryQuery) o);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return partialHashCode();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -144,10 +144,9 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
|||||||
{
|
{
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
TimeseriesQuery that = (TimeseriesQuery) o;
|
TimeseriesQuery that = (TimeseriesQuery) o;
|
||||||
if (!partialEquals(that))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null)
|
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(that.aggregatorSpecs) : that.aggregatorSpecs != null)
|
||||||
return false;
|
return false;
|
||||||
@ -162,8 +161,8 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
|
|||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = dimFilter != null ? dimFilter.hashCode() : 0;
|
int result = super.hashCode();
|
||||||
result = 31 * result + partialHashCode();
|
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
||||||
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
result = 31 * result + (granularity != null ? granularity.hashCode() : 0);
|
||||||
result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0);
|
result = 31 * result + (aggregatorSpecs != null ? aggregatorSpecs.hashCode() : 0);
|
||||||
result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0);
|
result = 31 * result + (postAggregatorSpecs != null ? postAggregatorSpecs.hashCode() : 0);
|
||||||
|
@ -212,10 +212,9 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
|
|||||||
{
|
{
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
if (!super.equals(o)) return false;
|
||||||
|
|
||||||
TopNQuery topNQuery = (TopNQuery) o;
|
TopNQuery topNQuery = (TopNQuery) o;
|
||||||
if (!partialEquals(topNQuery))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (threshold != topNQuery.threshold) return false;
|
if (threshold != topNQuery.threshold) return false;
|
||||||
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(topNQuery.aggregatorSpecs) : topNQuery.aggregatorSpecs != null)
|
if (aggregatorSpecs != null ? !aggregatorSpecs.equals(topNQuery.aggregatorSpecs) : topNQuery.aggregatorSpecs != null)
|
||||||
@ -235,8 +234,8 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
|
|||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
int result = dimensionSpec != null ? dimensionSpec.hashCode() : 0;
|
int result = super.hashCode();
|
||||||
result = 31 * result + partialHashCode();
|
result = 31 * result + (dimensionSpec != null ? dimensionSpec.hashCode() : 0);
|
||||||
result = 31 * result + (topNMetricSpec != null ? topNMetricSpec.hashCode() : 0);
|
result = 31 * result + (topNMetricSpec != null ? topNMetricSpec.hashCode() : 0);
|
||||||
result = 31 * result + threshold;
|
result = 31 * result + threshold;
|
||||||
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
result = 31 * result + (dimFilter != null ? dimFilter.hashCode() : 0);
|
||||||
|
Loading…
x
Reference in New Issue
Block a user