diff --git a/docs/content/querying/post-aggregations.md b/docs/content/querying/post-aggregations.md index 4e521305df6..c081a7dea0a 100644 --- a/docs/content/querying/post-aggregations.md +++ b/docs/content/querying/post-aggregations.md @@ -54,6 +54,26 @@ The constant post-aggregator always returns the specified value. { "type" : "constant", "name" : , "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" : , + "fields": [, , ...] +} +``` + ### 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. diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 9d18ba252fa..353f9ddc5cc 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -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 { diff --git a/processing/src/main/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregator.java new file mode 100644 index 00000000000..8d9a1626c27 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregator.java @@ -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 fields; + + @JsonCreator + public DoubleGreatestPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fields") List 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 getDependentFields() + { + Set dependentFields = Sets.newHashSet(); + for (PostAggregator field : fields) { + dependentFields.addAll(field.getDependentFields()); + } + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Object compute(Map values) + { + Iterator 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 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; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/post/DoubleLeastPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/DoubleLeastPostAggregator.java new file mode 100644 index 00000000000..c0128d3c242 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/post/DoubleLeastPostAggregator.java @@ -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 fields; + + @JsonCreator + public DoubleLeastPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fields") List 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 getDependentFields() + { + Set dependentFields = Sets.newHashSet(); + for (PostAggregator field : fields) { + dependentFields.addAll(field.getDependentFields()); + } + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Object compute(Map values) + { + Iterator 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 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; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/post/LongGreatestPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/LongGreatestPostAggregator.java new file mode 100644 index 00000000000..413621ce2f8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/post/LongGreatestPostAggregator.java @@ -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 fields; + + @JsonCreator + public LongGreatestPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fields") List 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 getDependentFields() + { + Set dependentFields = Sets.newHashSet(); + for (PostAggregator field : fields) { + dependentFields.addAll(field.getDependentFields()); + } + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Object compute(Map values) + { + Iterator 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 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; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/post/LongLeastPostAggregator.java b/processing/src/main/java/io/druid/query/aggregation/post/LongLeastPostAggregator.java new file mode 100644 index 00000000000..cd7701a6365 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/post/LongLeastPostAggregator.java @@ -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 fields; + + @JsonCreator + public LongLeastPostAggregator( + @JsonProperty("name") String name, + @JsonProperty("fields") List 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 getDependentFields() + { + Set dependentFields = Sets.newHashSet(); + for (PostAggregator field : fields) { + dependentFields.addAll(field.getDependentFields()); + } + return dependentFields; + } + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Override + public Object compute(Map values) + { + Iterator 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 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; + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java new file mode 100644 index 00000000000..8f3068cf80c --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java @@ -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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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)); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java new file mode 100644 index 00000000000..d139fb448c9 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java @@ -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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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)); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java new file mode 100644 index 00000000000..f9597146d62 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java @@ -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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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)); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/post/LongLeastPostAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/post/LongLeastPostAggregatorTest.java new file mode 100644 index 00000000000..fc487ab559c --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/post/LongLeastPostAggregatorTest.java @@ -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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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 metricValues = new HashMap(); + metricValues.put(aggName, agg.get()); + + List 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)); + } +}