Add Greatest and Least post aggregations (#3567)

This commit is contained in:
Erik Dubbelboer 2016-12-08 02:58:23 +01:00 committed by Jonathan Wei
parent 943982b7b0
commit bb9e35e1af
10 changed files with 999 additions and 1 deletions

View File

@ -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.

View File

@ -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
{

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}