mirror of https://github.com/apache/druid.git
Add Greatest and Least post aggregations (#3567)
This commit is contained in:
parent
943982b7b0
commit
bb9e35e1af
|
@ -54,6 +54,26 @@ The constant post-aggregator always returns the specified value.
|
|||
{ "type" : "constant", "name" : <output_name>, "value" : <numerical_value> }
|
||||
```
|
||||
|
||||
### Greatest / Least post-aggregators
|
||||
|
||||
`doubleGreatest` and `longGreatest` computes the maximum of all fields and Double.NEGATIVE_INFINITY.
|
||||
`doubleLeast` and `longLeast` computes the minimum of all fields and Double.POSITIVE_INFINITY.
|
||||
|
||||
The difference between the `doubleMax` aggregator and the `doubleGreatest` post-aggregator is that `doubleMax` returns the highest value of
|
||||
all rows for one specific column while `doubleGreatest` returns the highest value of multiple columns in one row. These are similar to the
|
||||
SQL [MAX](https://dev.mysql.com/doc/refman/5.7/en/group-by-functions.html#function_max) and
|
||||
[GREATEST](shttp://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#function_greatest) functions.
|
||||
|
||||
Example:
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "doubleGreatest",
|
||||
"name" : <output_name>,
|
||||
"fields": [<post_aggregator>, <post_aggregator>, ...]
|
||||
}
|
||||
```
|
||||
|
||||
### JavaScript post-aggregator
|
||||
|
||||
Applies the provided JavaScript function to the given fields. Fields are passed as arguments to the JavaScript function in the given order.
|
||||
|
|
|
@ -41,9 +41,13 @@ import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
|||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||
import io.druid.query.aggregation.post.DoubleGreatestPostAggregator;
|
||||
import io.druid.query.aggregation.post.DoubleLeastPostAggregator;
|
||||
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
|
||||
import io.druid.query.aggregation.post.JavaScriptPostAggregator;
|
||||
import io.druid.query.aggregation.post.ExpressionPostAggregator;
|
||||
import io.druid.query.aggregation.post.LongGreatestPostAggregator;
|
||||
import io.druid.query.aggregation.post.LongLeastPostAggregator;
|
||||
import io.druid.segment.serde.ComplexMetrics;
|
||||
|
||||
/**
|
||||
|
@ -88,7 +92,11 @@ public class AggregatorsModule extends SimpleModule
|
|||
@JsonSubTypes.Type(name = "fieldAccess", value = FieldAccessPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "constant", value = ConstantPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "javascript", value = JavaScriptPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "hyperUniqueCardinality", value = HyperUniqueFinalizingPostAggregator.class)
|
||||
@JsonSubTypes.Type(name = "hyperUniqueCardinality", value = HyperUniqueFinalizingPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "doubleGreatest", value = DoubleGreatestPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "doubleLeast", value = DoubleLeastPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "longGreatest", value = LongGreatestPostAggregator.class),
|
||||
@JsonSubTypes.Type(name = "longLeast", value = LongLeastPostAggregator.class)
|
||||
})
|
||||
public static interface PostAggregatorMixin
|
||||
{
|
||||
|
|
|
@ -0,0 +1,146 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class DoubleGreatestPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return ((Double) o).compareTo((Double) o1);
|
||||
}
|
||||
};
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
||||
@JsonCreator
|
||||
public DoubleGreatestPostAggregator(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fields") List<PostAggregator> fields
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(fields != null && fields.size() > 0, "Illegal number of fields[%s], must be > 0");
|
||||
|
||||
this.name = name;
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
for (PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
return dependentFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
double retVal = Double.NEGATIVE_INFINITY;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
double other = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
if (other > retVal) {
|
||||
retVal = other;
|
||||
}
|
||||
}
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<PostAggregator> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DoubleGreatestPostAggregator{" +
|
||||
"name='" + name + '\'' +
|
||||
", fields=" + fields +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DoubleGreatestPostAggregator that = (DoubleGreatestPostAggregator) o;
|
||||
|
||||
if (!fields.equals(that.fields)) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = name != null ? name.hashCode() : 0;
|
||||
result = 31 * result + fields.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,146 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class DoubleLeastPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return ((Double) o).compareTo((Double) o1);
|
||||
}
|
||||
};
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
||||
@JsonCreator
|
||||
public DoubleLeastPostAggregator(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fields") List<PostAggregator> fields
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(fields != null && fields.size() > 0, "Illegal number of fields[%s], must be > 0");
|
||||
|
||||
this.name = name;
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
for (PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
return dependentFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
double retVal = Double.POSITIVE_INFINITY;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
double other = ((Number) fieldsIter.next().compute(values)).doubleValue();
|
||||
if (other < retVal) {
|
||||
retVal = other;
|
||||
}
|
||||
}
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<PostAggregator> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DoubleLeastPostAggregator{" +
|
||||
"name='" + name + '\'' +
|
||||
", fields=" + fields +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
DoubleLeastPostAggregator that = (DoubleLeastPostAggregator) o;
|
||||
|
||||
if (!fields.equals(that.fields)) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = name != null ? name.hashCode() : 0;
|
||||
result = 31 * result + fields.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class LongGreatestPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return Longs.compare(((Number) o).longValue(), ((Number) o1).longValue());
|
||||
}
|
||||
};
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
||||
@JsonCreator
|
||||
public LongGreatestPostAggregator(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fields") List<PostAggregator> fields
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(fields != null && fields.size() > 0, "Illegal number of fields[%s], must be > 0");
|
||||
|
||||
this.name = name;
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
for (PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
return dependentFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
long retVal = Long.MIN_VALUE;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
long other = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
if (other > retVal) {
|
||||
retVal = other;
|
||||
}
|
||||
}
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<PostAggregator> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "LongGreatestPostAggregator{" +
|
||||
"name='" + name + '\'' +
|
||||
", fields=" + fields +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
LongGreatestPostAggregator that = (LongGreatestPostAggregator) o;
|
||||
|
||||
if (!fields.equals(that.fields)) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = name != null ? name.hashCode() : 0;
|
||||
result = 31 * result + fields.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,147 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
|
||||
public class LongLeastPostAggregator implements PostAggregator
|
||||
{
|
||||
private static final Comparator COMPARATOR = new Comparator()
|
||||
{
|
||||
@Override
|
||||
public int compare(Object o, Object o1)
|
||||
{
|
||||
return Longs.compare(((Number) o).longValue(), ((Number) o1).longValue());
|
||||
}
|
||||
};
|
||||
|
||||
private final String name;
|
||||
private final List<PostAggregator> fields;
|
||||
|
||||
@JsonCreator
|
||||
public LongLeastPostAggregator(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fields") List<PostAggregator> fields
|
||||
)
|
||||
{
|
||||
Preconditions.checkArgument(fields != null && fields.size() > 0, "Illegal number of fields[%s], must be > 0");
|
||||
|
||||
this.name = name;
|
||||
this.fields = fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
for (PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
return dependentFields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object compute(Map<String, Object> values)
|
||||
{
|
||||
Iterator<PostAggregator> fieldsIter = fields.iterator();
|
||||
long retVal = Long.MAX_VALUE;
|
||||
if (fieldsIter.hasNext()) {
|
||||
retVal = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
while (fieldsIter.hasNext()) {
|
||||
long other = ((Number) fieldsIter.next().compute(values)).longValue();
|
||||
if (other < retVal) {
|
||||
retVal = other;
|
||||
}
|
||||
}
|
||||
}
|
||||
return retVal;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<PostAggregator> getFields()
|
||||
{
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "LongLeastPostAggregator{" +
|
||||
"name='" + name + '\'' +
|
||||
", fields=" + fields +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
LongLeastPostAggregator that = (LongLeastPostAggregator) o;
|
||||
|
||||
if (!fields.equals(that.fields)) {
|
||||
return false;
|
||||
}
|
||||
if (name != null ? !name.equals(that.name) : that.name != null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
int result = name != null ? name.hashCode() : 0;
|
||||
result = 31 * result + fields.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.aggregation.CountAggregator;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DoubleGreatestPostAggregatorTest
|
||||
{
|
||||
@Test
|
||||
public void testCompute()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
DoubleGreatestPostAggregator greatestPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 6D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
greatestPostAggregator = new DoubleGreatestPostAggregator("greatest", postAggregatorList);
|
||||
Assert.assertEquals(6.0, greatestPostAggregator.compute(metricValues));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComparator()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
DoubleGreatestPostAggregator greatestPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 2D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
greatestPostAggregator = new DoubleGreatestPostAggregator("greatest", postAggregatorList);
|
||||
Comparator comp = greatestPostAggregator.getComparator();
|
||||
Object before = greatestPostAggregator.compute(metricValues);
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
metricValues.put(aggName, agg.get());
|
||||
Object after = greatestPostAggregator.compute(metricValues);
|
||||
|
||||
Assert.assertEquals(-1, comp.compare(before, after));
|
||||
Assert.assertEquals(0, comp.compare(before, before));
|
||||
Assert.assertEquals(0, comp.compare(after, after));
|
||||
Assert.assertEquals(1, comp.compare(after, before));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.aggregation.CountAggregator;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class DoubleLeastPostAggregatorTest
|
||||
{
|
||||
@Test
|
||||
public void testCompute()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
DoubleLeastPostAggregator leastPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 6D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
leastPostAggregator = new DoubleLeastPostAggregator("least", postAggregatorList);
|
||||
Assert.assertEquals(3.0, leastPostAggregator.compute(metricValues));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComparator()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
DoubleLeastPostAggregator leastPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 2D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
leastPostAggregator = new DoubleLeastPostAggregator("least", postAggregatorList);
|
||||
Comparator comp = leastPostAggregator.getComparator();
|
||||
Object before = leastPostAggregator.compute(metricValues);
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
metricValues.put(aggName, agg.get());
|
||||
Object after = leastPostAggregator.compute(metricValues);
|
||||
|
||||
Assert.assertEquals(-1, comp.compare(before, after));
|
||||
Assert.assertEquals(0, comp.compare(before, before));
|
||||
Assert.assertEquals(0, comp.compare(after, after));
|
||||
Assert.assertEquals(1, comp.compare(after, before));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.aggregation.CountAggregator;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongGreatestPostAggregatorTest
|
||||
{
|
||||
@Test
|
||||
public void testCompute()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
LongGreatestPostAggregator greatestPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 6D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
greatestPostAggregator = new LongGreatestPostAggregator("greatest", postAggregatorList);
|
||||
Assert.assertEquals(Long.valueOf(6), greatestPostAggregator.compute(metricValues));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComparator()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
LongGreatestPostAggregator greatestPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 2D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
greatestPostAggregator = new LongGreatestPostAggregator("greatest", postAggregatorList);
|
||||
Comparator comp = greatestPostAggregator.getComparator();
|
||||
Object before = greatestPostAggregator.compute(metricValues);
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
metricValues.put(aggName, agg.get());
|
||||
Object after = greatestPostAggregator.compute(metricValues);
|
||||
|
||||
Assert.assertEquals(-1, comp.compare(before, after));
|
||||
Assert.assertEquals(0, comp.compare(before, before));
|
||||
Assert.assertEquals(0, comp.compare(after, after));
|
||||
Assert.assertEquals(1, comp.compare(after, before));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,96 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.post;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.aggregation.CountAggregator;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class LongLeastPostAggregatorTest
|
||||
{
|
||||
@Test
|
||||
public void testCompute()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
LongLeastPostAggregator leastPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 6D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
leastPostAggregator = new LongLeastPostAggregator("leastPostAggregator", postAggregatorList);
|
||||
Assert.assertEquals(Long.valueOf(3), leastPostAggregator.compute(metricValues));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComparator()
|
||||
{
|
||||
final String aggName = "rows";
|
||||
LongLeastPostAggregator leastPostAggregator;
|
||||
CountAggregator agg = new CountAggregator();
|
||||
Map<String, Object> metricValues = new HashMap<String, Object>();
|
||||
metricValues.put(aggName, agg.get());
|
||||
|
||||
List<PostAggregator> postAggregatorList =
|
||||
Lists.newArrayList(
|
||||
new ConstantPostAggregator(
|
||||
"roku", 2D
|
||||
),
|
||||
new FieldAccessPostAggregator(
|
||||
"rows", aggName
|
||||
)
|
||||
);
|
||||
|
||||
leastPostAggregator = new LongLeastPostAggregator("leastPostAggregator", postAggregatorList);
|
||||
Comparator comp = leastPostAggregator.getComparator();
|
||||
Object before = leastPostAggregator.compute(metricValues);
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
agg.aggregate();
|
||||
metricValues.put(aggName, agg.get());
|
||||
Object after = leastPostAggregator.compute(metricValues);
|
||||
|
||||
Assert.assertEquals(-1, comp.compare(before, after));
|
||||
Assert.assertEquals(0, comp.compare(before, before));
|
||||
Assert.assertEquals(0, comp.compare(after, after));
|
||||
Assert.assertEquals(1, comp.compare(after, before));
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue