From 00436f93e2402f80f5899350fc7149e023d6b878 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Sun, 10 May 2015 10:30:46 -0500 Subject: [PATCH 1/5] long max/min aggregators implementation --- .../io/druid/jackson/AggregatorsModule.java | 4 + .../query/aggregation/LongMaxAggregator.java | 97 ++++++++++ .../aggregation/LongMaxAggregatorFactory.java | 179 ++++++++++++++++++ .../aggregation/LongMaxBufferAggregator.java | 72 +++++++ .../query/aggregation/LongMinAggregator.java | 97 ++++++++++ .../aggregation/LongMinAggregatorFactory.java | 179 ++++++++++++++++++ .../aggregation/LongMinBufferAggregator.java | 73 +++++++ 7 files changed, 701 insertions(+) create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java create mode 100644 processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index db7c75201c9..6fce1c3d437 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -27,6 +27,8 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; import io.druid.query.aggregation.HistogramAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; +import io.druid.query.aggregation.LongMaxAggregatorFactory; +import io.druid.query.aggregation.LongMinAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.aggregation.MinAggregatorFactory; @@ -64,6 +66,8 @@ public class AggregatorsModule extends SimpleModule @JsonSubTypes.Type(name = "doubleSum", value = DoubleSumAggregatorFactory.class), @JsonSubTypes.Type(name = "max", value = MaxAggregatorFactory.class), @JsonSubTypes.Type(name = "min", value = MinAggregatorFactory.class), + @JsonSubTypes.Type(name = "longMax", value = LongMaxAggregatorFactory.class), + @JsonSubTypes.Type(name = "longMin", value = LongMinAggregatorFactory.class), @JsonSubTypes.Type(name = "javascript", value = JavaScriptAggregatorFactory.class), @JsonSubTypes.Type(name = "histogram", value = HistogramAggregatorFactory.class), @JsonSubTypes.Type(name = "hyperUnique", value = HyperUniquesAggregatorFactory.class), diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java new file mode 100644 index 00000000000..a3fefea2b8d --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregator.java @@ -0,0 +1,97 @@ +/* +* 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; + +import io.druid.segment.LongColumnSelector; + +import java.util.Comparator; + +/** + */ +public class LongMaxAggregator implements Aggregator +{ + static final Comparator COMPARATOR = LongSumAggregator.COMPARATOR; + + static long combineValues(Object lhs, Object rhs) + { + return Math.max(((Number) lhs).longValue(), ((Number) rhs).longValue()); + } + + private final LongColumnSelector selector; + private final String name; + + private long max; + + public LongMaxAggregator(String name, LongColumnSelector selector) + { + this.name = name; + this.selector = selector; + + reset(); + } + + @Override + public void aggregate() + { + max = Math.max(max, selector.get()); + } + + @Override + public void reset() + { + max = Long.MIN_VALUE; + } + + @Override + public Object get() + { + return max; + } + + @Override + public float getFloat() + { + return (float) max; + } + + @Override + public long getLong() + { + return max; + } + + @Override + public String getName() + { + return this.name; + } + + @Override + public Aggregator clone() + { + return new LongMaxAggregator(name, selector); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java new file mode 100644 index 00000000000..c71baf9a3dc --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxAggregatorFactory.java @@ -0,0 +1,179 @@ +/* +* 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Longs; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class LongMaxAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0xA; + + private final String fieldName; + private final String name; + + @JsonCreator + public LongMaxAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new LongMaxAggregator(name, metricFactory.makeLongColumnSelector(fieldName)); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new LongMaxBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return LongMaxAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return LongMaxAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new LongMaxAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new LongMaxAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "long"; + } + + @Override + public int getMaxIntermediateSize() + { + return Longs.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Long.MIN_VALUE; + } + + @Override + public String toString() + { + return "LongMaxAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LongMaxAggregatorFactory that = (LongMaxAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java new file mode 100644 index 00000000000..daa92627396 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMaxBufferAggregator.java @@ -0,0 +1,72 @@ +/* +* 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; + +import io.druid.segment.LongColumnSelector; + +import java.nio.ByteBuffer; + +/** + */ +public class LongMaxBufferAggregator implements BufferAggregator +{ + private final LongColumnSelector selector; + + public LongMaxBufferAggregator(LongColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + buf.putLong(position, Long.MIN_VALUE); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putLong(position, Math.max(buf.getLong(position), selector.get())); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return (float) buf.getLong(position); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java new file mode 100644 index 00000000000..257a863d7aa --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregator.java @@ -0,0 +1,97 @@ +/* +* 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; + +import io.druid.segment.LongColumnSelector; + +import java.util.Comparator; + +/** + */ +public class LongMinAggregator implements Aggregator +{ + static final Comparator COMPARATOR = LongSumAggregator.COMPARATOR; + + static long combineValues(Object lhs, Object rhs) + { + return Math.min(((Number) lhs).longValue(), ((Number) rhs).longValue()); + } + + private final LongColumnSelector selector; + private final String name; + + private long min; + + public LongMinAggregator(String name, LongColumnSelector selector) + { + this.name = name; + this.selector = selector; + + reset(); + } + + @Override + public void aggregate() + { + min = Math.min(min, selector.get()); + } + + @Override + public void reset() + { + min = Long.MAX_VALUE; + } + + @Override + public Object get() + { + return min; + } + + @Override + public float getFloat() + { + return (float) min; + } + + @Override + public long getLong() + { + return min; + } + + @Override + public String getName() + { + return this.name; + } + + @Override + public Aggregator clone() + { + return new LongMinAggregator(name, selector); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java new file mode 100644 index 00000000000..7f7eca7989f --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinAggregatorFactory.java @@ -0,0 +1,179 @@ +/* +* 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Longs; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class LongMinAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0xB; + + private final String fieldName; + private final String name; + + @JsonCreator + public LongMinAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new LongMinAggregator(name, metricFactory.makeLongColumnSelector(fieldName)); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new LongMinBufferAggregator(metricFactory.makeLongColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return LongMinAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return LongMinAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new LongMinAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new LongMinAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "long"; + } + + @Override + public int getMaxIntermediateSize() + { + return Longs.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Long.MAX_VALUE; + } + + @Override + public String toString() + { + return "LongMinAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LongMinAggregatorFactory that = (LongMinAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java new file mode 100644 index 00000000000..382dbe14afa --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/LongMinBufferAggregator.java @@ -0,0 +1,73 @@ +/* +* 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; + +import io.druid.segment.LongColumnSelector; + +import java.nio.ByteBuffer; + +/** + */ +public class LongMinBufferAggregator implements BufferAggregator +{ + private final LongColumnSelector selector; + + public LongMinBufferAggregator(LongColumnSelector selector) + { + this.selector = selector; + } + + @Override + public void init(ByteBuffer buf, int position) + { + buf.putLong(position, Long.MAX_VALUE); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + buf.putLong(position, Math.min(buf.getLong(position), selector.get())); + } + + @Override + public Object get(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + return (float) buf.getLong(position); + } + + + @Override + public long getLong(ByteBuffer buf, int position) + { + return buf.getLong(position); + } + + @Override + public void close() + { + // no resources to cleanup + } +} From 2de38f7d2975acdd23ff1c9a60bf5529dc6c7bd2 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 11 May 2015 10:08:42 -0500 Subject: [PATCH 2/5] UTs for long[Max/Min] aggregation --- .../aggregation/LongMaxAggregationTest.java | 125 ++++++++++++++++++ .../aggregation/LongMinAggregationTest.java | 125 ++++++++++++++++++ 2 files changed, 250 insertions(+) create mode 100644 processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java diff --git a/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java new file mode 100644 index 00000000000..48fba38727b --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/LongMaxAggregationTest.java @@ -0,0 +1,125 @@ +/* +* 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; + +import com.google.common.primitives.Longs; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.ColumnSelectorFactory; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + */ +public class LongMaxAggregationTest +{ + private LongMaxAggregatorFactory longMaxAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestLongColumnSelector selector; + + private long[] values = {9223372036854775802l, 9223372036854775803l, 9223372036854775806l, 9223372036854775805l}; + + public LongMaxAggregationTest() throws Exception + { + String aggSpecJson = "{\"type\": \"longMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + longMaxAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , LongMaxAggregatorFactory.class); + } + + @Before + public void setup() + { + selector = new TestLongColumnSelector(values); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeLongColumnSelector("nilly")).andReturn(selector); + EasyMock.replay(colSelectorFactory); + } + + @Test + public void testLongMaxAggregator() + { + LongMaxAggregator agg = (LongMaxAggregator)longMaxAggFactory.factorize(colSelectorFactory); + + Assert.assertEquals("billy", agg.getName()); + + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + + Assert.assertEquals(values[2], ((Long) agg.get()).longValue()); + Assert.assertEquals(values[2], agg.getLong()); + Assert.assertEquals((float) values[2], agg.getFloat(), 0.0001); + + agg.reset(); + Assert.assertEquals(Long.MIN_VALUE, agg.getLong()); + } + + @Test + public void testLongMaxBufferAggregator() + { + LongMaxBufferAggregator agg = (LongMaxBufferAggregator)longMaxAggFactory.factorizeBuffered(colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[Longs.BYTES]); + agg.init(buffer, 0); + + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + + Assert.assertEquals(values[2], ((Long) agg.get(buffer, 0)).longValue()); + Assert.assertEquals(values[2], agg.getLong(buffer, 0)); + Assert.assertEquals((float)values[2], agg.getFloat(buffer, 0), 0.0001); + } + + @Test + public void testCombine() + { + Assert.assertEquals(9223372036854775803l, longMaxAggFactory.combine(9223372036854775800l, 9223372036854775803l)); + } + + @Test + public void testEqualsAndHashCode() throws Exception + { + LongMaxAggregatorFactory one = new LongMaxAggregatorFactory("name1", "fieldName1"); + LongMaxAggregatorFactory oneMore = new LongMaxAggregatorFactory("name1", "fieldName1"); + LongMaxAggregatorFactory two = new LongMaxAggregatorFactory("name2", "fieldName2"); + + Assert.assertEquals(one.hashCode(), oneMore.hashCode()); + + Assert.assertTrue(one.equals(oneMore)); + Assert.assertFalse(one.equals(two)); + } + + private void aggregate(TestLongColumnSelector selector, LongMaxAggregator agg) + { + agg.aggregate(); + selector.increment(); + } + + private void aggregate(TestLongColumnSelector selector, LongMaxBufferAggregator agg, ByteBuffer buff, int position) + { + agg.aggregate(buff, position); + selector.increment(); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java new file mode 100644 index 00000000000..1b06df2e279 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/LongMinAggregationTest.java @@ -0,0 +1,125 @@ +/* +* 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; + +import com.google.common.primitives.Longs; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.ColumnSelectorFactory; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + */ +public class LongMinAggregationTest +{ + private LongMinAggregatorFactory longMinAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestLongColumnSelector selector; + + private long[] values = {-9223372036854775802l, -9223372036854775803l, -9223372036854775806l, -9223372036854775805l}; + + public LongMinAggregationTest() throws Exception + { + String aggSpecJson = "{\"type\": \"longMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + longMinAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , LongMinAggregatorFactory.class); + } + + @Before + public void setup() + { + selector = new TestLongColumnSelector(values); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeLongColumnSelector("nilly")).andReturn(selector); + EasyMock.replay(colSelectorFactory); + } + + @Test + public void testLongMinAggregator() + { + LongMinAggregator agg = (LongMinAggregator)longMinAggFactory.factorize(colSelectorFactory); + + Assert.assertEquals("billy", agg.getName()); + + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + + Assert.assertEquals(values[2], ((Long) agg.get()).longValue()); + Assert.assertEquals(values[2], agg.getLong()); + Assert.assertEquals((float) values[2], agg.getFloat(), 0.0001); + + agg.reset(); + Assert.assertEquals(Long.MAX_VALUE, agg.getLong()); + } + + @Test + public void testLongMinBufferAggregator() + { + LongMinBufferAggregator agg = (LongMinBufferAggregator)longMinAggFactory.factorizeBuffered(colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[Longs.BYTES]); + agg.init(buffer, 0); + + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + + Assert.assertEquals(values[2], ((Long) agg.get(buffer, 0)).longValue()); + Assert.assertEquals(values[2], agg.getLong(buffer, 0)); + Assert.assertEquals((float)values[2], agg.getFloat(buffer, 0), 0.0001); + } + + @Test + public void testCombine() + { + Assert.assertEquals(-9223372036854775803l, longMinAggFactory.combine(-9223372036854775800l, -9223372036854775803l)); + } + + @Test + public void testEqualsAndHashCode() throws Exception + { + LongMinAggregatorFactory one = new LongMinAggregatorFactory("name1", "fieldName1"); + LongMinAggregatorFactory oneMore = new LongMinAggregatorFactory("name1", "fieldName1"); + LongMinAggregatorFactory two = new LongMinAggregatorFactory("name2", "fieldName2"); + + Assert.assertEquals(one.hashCode(), oneMore.hashCode()); + + Assert.assertTrue(one.equals(oneMore)); + Assert.assertFalse(one.equals(two)); + } + + private void aggregate(TestLongColumnSelector selector, LongMinAggregator agg) + { + agg.aggregate(); + selector.increment(); + } + + private void aggregate(TestLongColumnSelector selector, LongMinBufferAggregator agg, ByteBuffer buff, int position) + { + agg.aggregate(buff, position); + selector.increment(); + } +} From d0ec94512909dc883081267967fac9fb0b4f8d70 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 12 May 2015 13:13:16 -0500 Subject: [PATCH 3/5] adding aliases doubleMax and doubleMin for max and min respectively renamed all [Max/Min]*.java to [DoubleMax/DoubleMin]*.java and created [Max/Min]AggregatorFactory.java which can be removed when we dont need the min/max aggregator type backward compatibility --- .../ApproximateHistogramTopNQueryTest.java | 8 +- .../io/druid/jackson/AggregatorsModule.java | 10 +- ...gregator.java => DoubleMaxAggregator.java} | 6 +- .../DoubleMaxAggregatorFactory.java | 181 +++++++++++++++++ ...or.java => DoubleMaxBufferAggregator.java} | 4 +- ...gregator.java => DoubleMinAggregator.java} | 6 +- .../DoubleMinAggregatorFactory.java | 181 +++++++++++++++++ ...or.java => DoubleMinBufferAggregator.java} | 4 +- .../aggregation/MaxAggregatorFactory.java | 187 ++--------------- .../aggregation/MinAggregatorFactory.java | 189 ++---------------- .../query/aggregation/AggregatorUtilTest.java | 4 +- ...Test.java => DoubleMaxAggregatorTest.java} | 6 +- ...Test.java => DoubleMinAggregatorTest.java} | 6 +- .../query/groupby/GroupByQueryRunnerTest.java | 8 +- .../timeseries/TimeseriesQueryRunnerTest.java | 8 +- .../query/topn/TopNQueryRunnerBenchmark.java | 8 +- .../druid/query/topn/TopNQueryRunnerTest.java | 36 ++-- .../io/druid/query/topn/TopNQueryTest.java | 8 +- .../druid/query/topn/TopNUnionQueryTest.java | 8 +- .../java/io/druid/segment/AppendTest.java | 24 +-- .../io/druid/segment/SchemalessTestFull.java | 20 +- .../druid/segment/SchemalessTestSimple.java | 12 +- .../antlr4/io/druid/sql/antlr4/DruidSQL.g4 | 8 +- 23 files changed, 503 insertions(+), 429 deletions(-) rename processing/src/main/java/io/druid/query/aggregation/{MaxAggregator.java => DoubleMaxAggregator.java} (90%) create mode 100644 processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java rename processing/src/main/java/io/druid/query/aggregation/{MaxBufferAggregator.java => DoubleMaxBufferAggregator.java} (92%) rename processing/src/main/java/io/druid/query/aggregation/{MinAggregator.java => DoubleMinAggregator.java} (90%) create mode 100644 processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java rename processing/src/main/java/io/druid/query/aggregation/{MinBufferAggregator.java => DoubleMinBufferAggregator.java} (92%) rename processing/src/test/java/io/druid/query/aggregation/{MaxAggregatorTest.java => DoubleMaxAggregatorTest.java} (86%) rename processing/src/test/java/io/druid/query/aggregation/{MinAggregatorTest.java => DoubleMinAggregatorTest.java} (86%) diff --git a/extensions/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index b4f5c212aa6..1d9a35da86e 100644 --- a/extensions/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -27,8 +27,8 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryBuilder; @@ -121,8 +121,8 @@ public class ApproximateHistogramTopNQueryTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index"), + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index"), factory ) ) diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 6fce1c3d437..e28c0cf71b6 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -23,6 +23,10 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.hash.Hashing; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; +import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; import io.druid.query.aggregation.HistogramAggregatorFactory; @@ -30,8 +34,6 @@ import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongMaxAggregatorFactory; import io.druid.query.aggregation.LongMinAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; @@ -64,6 +66,10 @@ public class AggregatorsModule extends SimpleModule @JsonSubTypes.Type(name = "count", value = CountAggregatorFactory.class), @JsonSubTypes.Type(name = "longSum", value = LongSumAggregatorFactory.class), @JsonSubTypes.Type(name = "doubleSum", value = DoubleSumAggregatorFactory.class), + @JsonSubTypes.Type(name = "doubleMax", value = DoubleMaxAggregatorFactory.class), + @JsonSubTypes.Type(name = "doubleMin", value = DoubleMinAggregatorFactory.class), + //Note: max/min are exactly same as doubleMax/doubleMin, max/min are still here only + //to be backward compatible with 0.7.2 and before @JsonSubTypes.Type(name = "max", value = MaxAggregatorFactory.class), @JsonSubTypes.Type(name = "min", value = MinAggregatorFactory.class), @JsonSubTypes.Type(name = "longMax", value = LongMaxAggregatorFactory.class), diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java similarity index 90% rename from processing/src/main/java/io/druid/query/aggregation/MaxAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java index 15faa0f8463..8735ac061e2 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregator.java @@ -23,7 +23,7 @@ import java.util.Comparator; /** */ -public class MaxAggregator implements Aggregator +public class DoubleMaxAggregator implements Aggregator { static final Comparator COMPARATOR = DoubleSumAggregator.COMPARATOR; @@ -37,7 +37,7 @@ public class MaxAggregator implements Aggregator private double max; - public MaxAggregator(String name, FloatColumnSelector selector) + public DoubleMaxAggregator(String name, FloatColumnSelector selector) { this.name = name; this.selector = selector; @@ -84,7 +84,7 @@ public class MaxAggregator implements Aggregator @Override public Aggregator clone() { - return new MaxAggregator(name, selector); + return new DoubleMaxAggregator(name, selector); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java new file mode 100644 index 00000000000..74acf283b74 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxAggregatorFactory.java @@ -0,0 +1,181 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Doubles; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class DoubleMaxAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x3; + + private final String fieldName; + private final String name; + + @JsonCreator + public DoubleMaxAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new DoubleMaxAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new DoubleMaxBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return DoubleMaxAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return DoubleMaxAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new DoubleMaxAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new DoubleMaxAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((String) object); + } + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "float"; + } + + @Override + public int getMaxIntermediateSize() + { + return Doubles.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Double.NEGATIVE_INFINITY; + } + + @Override + public String toString() + { + return "DoubleMaxAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DoubleMaxAggregatorFactory that = (DoubleMaxAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java similarity index 92% rename from processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java index 22a13ff0e45..bd3fbe4e86e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMaxBufferAggregator.java @@ -23,11 +23,11 @@ import java.nio.ByteBuffer; /** */ -public class MaxBufferAggregator implements BufferAggregator +public class DoubleMaxBufferAggregator implements BufferAggregator { private final FloatColumnSelector selector; - public MaxBufferAggregator(FloatColumnSelector selector) + public DoubleMaxBufferAggregator(FloatColumnSelector selector) { this.selector = selector; } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java similarity index 90% rename from processing/src/main/java/io/druid/query/aggregation/MinAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java index 30ef02240a6..c1e71b9eddc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregator.java @@ -23,7 +23,7 @@ import java.util.Comparator; /** */ -public class MinAggregator implements Aggregator +public class DoubleMinAggregator implements Aggregator { static final Comparator COMPARATOR = DoubleSumAggregator.COMPARATOR; @@ -37,7 +37,7 @@ public class MinAggregator implements Aggregator private double min; - public MinAggregator(String name, FloatColumnSelector selector) + public DoubleMinAggregator(String name, FloatColumnSelector selector) { this.name = name; this.selector = selector; @@ -84,7 +84,7 @@ public class MinAggregator implements Aggregator @Override public Aggregator clone() { - return new MinAggregator(name, selector); + return new DoubleMinAggregator(name, selector); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java new file mode 100644 index 00000000000..a246a4ccfc8 --- /dev/null +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinAggregatorFactory.java @@ -0,0 +1,181 @@ +/* + * Druid - a distributed column store. + * Copyright 2012 - 2015 Metamarkets Group Inc. + * + * Licensed 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Doubles; +import com.metamx.common.StringUtils; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; + +/** + */ +public class DoubleMinAggregatorFactory implements AggregatorFactory +{ + private static final byte CACHE_TYPE_ID = 0x4; + + private final String fieldName; + private final String name; + + @JsonCreator + public DoubleMinAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName + ) + { + Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); + Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); + + this.name = name; + this.fieldName = fieldName; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new DoubleMinAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new DoubleMinBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); + } + + @Override + public Comparator getComparator() + { + return DoubleMinAggregator.COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return DoubleMinAggregator.combineValues(lhs, rhs); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new DoubleMinAggregatorFactory(name, name); + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new DoubleMinAggregatorFactory(fieldName, fieldName)); + } + + @Override + public Object deserialize(Object object) + { + // handle "NaN" / "Infinity" values serialized as strings in JSON + if (object instanceof String) { + return Double.parseDouble((String) object); + } + return object; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + public List requiredFields() + { + return Arrays.asList(fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); + } + + @Override + public String getTypeName() + { + return "float"; + } + + @Override + public int getMaxIntermediateSize() + { + return Doubles.BYTES; + } + + @Override + public Object getAggregatorStartValue() + { + return Double.POSITIVE_INFINITY; + } + + @Override + public String toString() + { + return "DoubleMinAggregatorFactory{" + + "fieldName='" + fieldName + '\'' + + ", name='" + name + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) + { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DoubleMinAggregatorFactory that = (DoubleMinAggregatorFactory) o; + + if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; + if (name != null ? !name.equals(that.name) : that.name != null) return false; + + return true; + } + + @Override + public int hashCode() + { + int result = fieldName != null ? fieldName.hashCode() : 0; + result = 31 * result + (name != null ? name.hashCode() : 0); + return result; + } +} diff --git a/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java similarity index 92% rename from processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java index d70434ba29d..18e1cb64e59 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/DoubleMinBufferAggregator.java @@ -23,11 +23,11 @@ import java.nio.ByteBuffer; /** */ -public class MinBufferAggregator implements BufferAggregator +public class DoubleMinBufferAggregator implements BufferAggregator { private final FloatColumnSelector selector; - public MinBufferAggregator(FloatColumnSelector selector) + public DoubleMinBufferAggregator(FloatColumnSelector selector) { this.selector = selector; } diff --git a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java index e02fc04fc02..00fc80ef020 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MaxAggregatorFactory.java @@ -1,181 +1,36 @@ /* - * Druid - a distributed column store. - * Copyright 2012 - 2015 Metamarkets Group Inc. - * - * Licensed 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. - */ +* 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; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.primitives.Doubles; -import com.metamx.common.StringUtils; -import io.druid.segment.ColumnSelectorFactory; - -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; /** + * This class can be removed when we dont need "max" aggregator backward compatibility */ -public class MaxAggregatorFactory implements AggregatorFactory +public class MaxAggregatorFactory extends DoubleMaxAggregatorFactory { - private static final byte CACHE_TYPE_ID = 0x3; - - private final String fieldName; - private final String name; - - @JsonCreator public MaxAggregatorFactory( @JsonProperty("name") String name, - @JsonProperty("fieldName") final String fieldName + @JsonProperty("fieldName") String fieldName ) { - Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); - Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); - - this.name = name; - this.fieldName = fieldName; - } - - @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) - { - return new MaxAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); - } - - @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) - { - return new MaxBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); - } - - @Override - public Comparator getComparator() - { - return MaxAggregator.COMPARATOR; - } - - @Override - public Object combine(Object lhs, Object rhs) - { - return MaxAggregator.combineValues(lhs, rhs); - } - - @Override - public AggregatorFactory getCombiningFactory() - { - return new MaxAggregatorFactory(name, name); - } - - @Override - public List getRequiredColumns() - { - return Arrays.asList(new MaxAggregatorFactory(fieldName, fieldName)); - } - - @Override - public Object deserialize(Object object) - { - // handle "NaN" / "Infinity" values serialized as strings in JSON - if (object instanceof String) { - return Double.parseDouble((String) object); - } - return object; - } - - @Override - public Object finalizeComputation(Object object) - { - return object; - } - - @JsonProperty - public String getFieldName() - { - return fieldName; - } - - @Override - @JsonProperty - public String getName() - { - return name; - } - - @Override - public List requiredFields() - { - return Arrays.asList(fieldName); - } - - @Override - public byte[] getCacheKey() - { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - - return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); - } - - @Override - public String getTypeName() - { - return "float"; - } - - @Override - public int getMaxIntermediateSize() - { - return Doubles.BYTES; - } - - @Override - public Object getAggregatorStartValue() - { - return Double.NEGATIVE_INFINITY; - } - - @Override - public String toString() - { - return "MaxAggregatorFactory{" + - "fieldName='" + fieldName + '\'' + - ", name='" + name + '\'' + - '}'; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - MaxAggregatorFactory that = (MaxAggregatorFactory) o; - - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; - - return true; - } - - @Override - public int hashCode() - { - int result = fieldName != null ? fieldName.hashCode() : 0; - result = 31 * result + (name != null ? name.hashCode() : 0); - return result; + super(name, fieldName); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java index 53239b591e3..b10f800da89 100644 --- a/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/MinAggregatorFactory.java @@ -1,181 +1,36 @@ /* - * Druid - a distributed column store. - * Copyright 2012 - 2015 Metamarkets Group Inc. - * - * Licensed 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. - */ +* 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; -import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import com.google.common.primitives.Doubles; -import com.metamx.common.StringUtils; -import io.druid.segment.ColumnSelectorFactory; - -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; /** - */ -public class MinAggregatorFactory implements AggregatorFactory + * This class can be removed when we dont need "min" aggregator backward compatibility + */ +public class MinAggregatorFactory extends DoubleMinAggregatorFactory { - private static final byte CACHE_TYPE_ID = 0x4; - - private final String fieldName; - private final String name; - - @JsonCreator public MinAggregatorFactory( @JsonProperty("name") String name, - @JsonProperty("fieldName") final String fieldName + @JsonProperty("fieldName") String fieldName ) { - Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); - Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); - - this.name = name; - this.fieldName = fieldName; - } - - @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) - { - return new MinAggregator(name, metricFactory.makeFloatColumnSelector(fieldName)); - } - - @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) - { - return new MinBufferAggregator(metricFactory.makeFloatColumnSelector(fieldName)); - } - - @Override - public Comparator getComparator() - { - return MinAggregator.COMPARATOR; - } - - @Override - public Object combine(Object lhs, Object rhs) - { - return MinAggregator.combineValues(lhs, rhs); - } - - @Override - public AggregatorFactory getCombiningFactory() - { - return new MinAggregatorFactory(name, name); - } - - @Override - public List getRequiredColumns() - { - return Arrays.asList(new MinAggregatorFactory(fieldName, fieldName)); - } - - @Override - public Object deserialize(Object object) - { - // handle "NaN" / "Infinity" values serialized as strings in JSON - if (object instanceof String) { - return Double.parseDouble((String) object); - } - return object; - } - - @Override - public Object finalizeComputation(Object object) - { - return object; - } - - @JsonProperty - public String getFieldName() - { - return fieldName; - } - - @Override - @JsonProperty - public String getName() - { - return name; - } - - @Override - public List requiredFields() - { - return Arrays.asList(fieldName); - } - - @Override - public byte[] getCacheKey() - { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - - return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); - } - - @Override - public String getTypeName() - { - return "float"; - } - - @Override - public int getMaxIntermediateSize() - { - return Doubles.BYTES; - } - - @Override - public Object getAggregatorStartValue() - { - return Double.POSITIVE_INFINITY; - } - - @Override - public String toString() - { - return "MinAggregatorFactory{" + - "fieldName='" + fieldName + '\'' + - ", name='" + name + '\'' + - '}'; - } - - @Override - public boolean equals(Object o) - { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - MinAggregatorFactory that = (MinAggregatorFactory) o; - - if (fieldName != null ? !fieldName.equals(that.fieldName) : that.fieldName != null) return false; - if (name != null ? !name.equals(that.name) : that.name != null) return false; - - return true; - } - - @Override - public int hashCode() - { - int result = fieldName != null ? fieldName.hashCode() : 0; - result = 31 * result + (name != null ? name.hashCode() : 0); - return result; + super(name, fieldName); } } diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java index e779193789e..93a0c63e1c2 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregatorUtilTest.java @@ -114,8 +114,8 @@ public class AggregatorUtilTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ); diff --git a/processing/src/test/java/io/druid/query/aggregation/MaxAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java similarity index 86% rename from processing/src/test/java/io/druid/query/aggregation/MaxAggregatorTest.java rename to processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java index 093bf387eb5..b2dc4406957 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MaxAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java @@ -22,9 +22,9 @@ import org.junit.Test; /** */ -public class MaxAggregatorTest +public class DoubleMaxAggregatorTest { - private void aggregate(TestFloatColumnSelector selector, MaxAggregator agg) + private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg) { agg.aggregate(); selector.increment(); @@ -35,7 +35,7 @@ public class MaxAggregatorTest { final float[] values = {0.15f, 0.27f, 0.0f, 0.93f}; final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - MaxAggregator agg = new MaxAggregator("billy", selector); + DoubleMaxAggregator agg = new DoubleMaxAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); diff --git a/processing/src/test/java/io/druid/query/aggregation/MinAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java similarity index 86% rename from processing/src/test/java/io/druid/query/aggregation/MinAggregatorTest.java rename to processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java index 74e1f74cab4..4f74baf8c5d 100644 --- a/processing/src/test/java/io/druid/query/aggregation/MinAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java @@ -22,9 +22,9 @@ import org.junit.Test; /** */ -public class MinAggregatorTest +public class DoubleMinAggregatorTest { - private void aggregate(TestFloatColumnSelector selector, MinAggregator agg) + private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg) { agg.aggregate(); selector.increment(); @@ -35,7 +35,7 @@ public class MinAggregatorTest { final float[] values = {0.15f, 0.27f, 0.0f, 0.93f}; final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - MinAggregator agg = new MinAggregator("billy", selector); + DoubleMinAggregator agg = new DoubleMinAggregator("billy", selector); Assert.assertEquals("billy", agg.getName()); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index 70dbaa273d0..aae346c48cb 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -44,10 +44,10 @@ import io.druid.query.QueryToolChest; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.JavaScriptAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -1964,7 +1964,7 @@ public class GroupByQueryRunnerTest .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs( Arrays.asList( - new MaxAggregatorFactory("idx", "idx") + new DoubleMaxAggregatorFactory("idx", "idx") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -2002,7 +2002,7 @@ public class GroupByQueryRunnerTest .setQuerySegmentSpec(QueryRunnerTestHelper.secondOnly) .setAggregatorSpecs( Arrays.asList( - new MaxAggregatorFactory("idx", "idx") + new DoubleMaxAggregatorFactory("idx", "idx") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) @@ -2039,7 +2039,7 @@ public class GroupByQueryRunnerTest .setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird) .setAggregatorSpecs( Arrays.asList( - new MaxAggregatorFactory("idx", "idx") + new DoubleMaxAggregatorFactory("idx", "idx") ) ) .setGranularity(QueryRunnerTestHelper.dayGran) diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 023476c155d..6d6e148f4fd 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -32,8 +32,8 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.DimFilter; @@ -165,8 +165,8 @@ public class TimeseriesQueryRunnerTest .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) .build(); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java index 93ebb9da6ed..d9697d3b6dd 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerBenchmark.java @@ -28,8 +28,8 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.QueryableIndexSegment; @@ -71,8 +71,8 @@ public class TopNQueryRunnerBenchmark extends AbstractBenchmark Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index c88301158e5..68a14aa8fee 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -35,9 +35,8 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.LongSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -54,7 +53,6 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.Column; import org.joda.time.DateTime; import org.joda.time.Interval; -import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -66,11 +64,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; /** */ @@ -139,8 +135,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -203,8 +199,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -268,8 +264,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -2069,8 +2065,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -2154,8 +2150,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -2347,8 +2343,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -2396,8 +2392,8 @@ public class TopNQueryRunnerTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java index 9cfa6be0aa3..dc073ea7381 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryTest.java @@ -23,8 +23,8 @@ import com.google.common.collect.Lists; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Query; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import org.junit.Assert; import org.junit.Test; @@ -59,8 +59,8 @@ public class TopNQueryTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java index 7dcd16313b4..4daa9efd119 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNUnionQueryTest.java @@ -27,8 +27,8 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.segment.TestHelper; import org.joda.time.DateTime; @@ -108,8 +108,8 @@ public class TopNUnionQueryTest Iterables.concat( QueryRunnerTestHelper.commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index f0387df2702..d604b3fcd94 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -28,9 +28,9 @@ import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; @@ -563,8 +563,8 @@ public class AppendTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -587,8 +587,8 @@ public class AppendTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -623,8 +623,8 @@ public class AppendTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -647,8 +647,8 @@ public class AppendTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -686,8 +686,8 @@ public class AppendTest Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index fac3b7c5bdb..70a38ddbcab 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -29,9 +29,9 @@ import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; @@ -1438,8 +1438,8 @@ public class SchemalessTestFull Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -1472,8 +1472,8 @@ public class SchemalessTestFull Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -1505,8 +1505,8 @@ public class SchemalessTestFull Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -1539,8 +1539,8 @@ public class SchemalessTestFull Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index 43c1519344f..77fb21fa1d9 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -27,9 +27,9 @@ import io.druid.query.Result; import io.druid.query.TestQueryRunners; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; @@ -138,8 +138,8 @@ public class SchemalessTestSimple Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) @@ -184,8 +184,8 @@ public class SchemalessTestSimple Iterables.concat( commonAggregators, Lists.newArrayList( - new MaxAggregatorFactory("maxIndex", "index"), - new MinAggregatorFactory("minIndex", "index") + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") ) ) ) diff --git a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 index 1cafead058d..f86a252fdb6 100644 --- a/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 +++ b/server/src/main/antlr4/io/druid/sql/antlr4/DruidSQL.g4 @@ -8,8 +8,8 @@ import io.druid.granularity.QueryGranularity; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; -import io.druid.query.aggregation.MaxAggregatorFactory; -import io.druid.query.aggregation.MinAggregatorFactory; +import io.druid.query.aggregation.DoubleMaxAggregatorFactory; +import io.druid.query.aggregation.DoubleMinAggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; @@ -70,8 +70,8 @@ import java.util.Map; AggregatorFactory evalAgg(String name, int fn) { switch (fn) { case SUM: return new DoubleSumAggregatorFactory("sum("+name+")", name); - case MIN: return new MinAggregatorFactory("min("+name+")", name); - case MAX: return new MaxAggregatorFactory("max("+name+")", name); + case MIN: return new DoubleMinAggregatorFactory("min("+name+")", name); + case MAX: return new DoubleMaxAggregatorFactory("max("+name+")", name); case COUNT: return new CountAggregatorFactory(name); } throw new IllegalArgumentException("Unknown function [" + fn + "]"); From cebb550796acdea004169670f388b83fb732472a Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 12 May 2015 23:37:55 -0500 Subject: [PATCH 4/5] additional UTs for [DoubleMax/DoubleMin] aggregation --- .../aggregation/DoubleMaxAggregationTest.java | 133 ++++++++++++++++++ .../aggregation/DoubleMaxAggregatorTest.java | 49 ------- .../aggregation/DoubleMinAggregationTest.java | 133 ++++++++++++++++++ .../aggregation/DoubleMinAggregatorTest.java | 49 ------- 4 files changed, 266 insertions(+), 98 deletions(-) create mode 100644 processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java delete mode 100644 processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java create mode 100644 processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java delete mode 100644 processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java new file mode 100644 index 00000000000..340802f1d73 --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -0,0 +1,133 @@ +/* +* 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; + +import com.google.common.primitives.Doubles; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.ColumnSelectorFactory; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + */ +public class DoubleMaxAggregationTest +{ + private DoubleMaxAggregatorFactory doubleMaxAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestFloatColumnSelector selector; + + private float[] values = {1.1f, 2.7f, 3.5f, 1.3f}; + + public DoubleMaxAggregationTest() throws Exception + { + String aggSpecJson = "{\"type\": \"doubleMax\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + doubleMaxAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMaxAggregatorFactory.class); + } + + @Before + public void setup() + { + selector = new TestFloatColumnSelector(values); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector); + EasyMock.replay(colSelectorFactory); + } + + @Test + public void testDoubleMaxAggregator() + { + DoubleMaxAggregator agg = (DoubleMaxAggregator) doubleMaxAggFactory.factorize(colSelectorFactory); + + Assert.assertEquals("billy", agg.getName()); + + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + + Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001); + Assert.assertEquals((long)values[2], agg.getLong()); + Assert.assertEquals(values[2], agg.getFloat(), 0.0001); + + agg.reset(); + Assert.assertEquals(Double.NEGATIVE_INFINITY, (Double) agg.get(), 0.0001); + } + + @Test + public void testDoubleMaxBufferAggregator() + { + DoubleMaxBufferAggregator agg = (DoubleMaxBufferAggregator) doubleMaxAggFactory.factorizeBuffered(colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[Doubles.BYTES]); + agg.init(buffer, 0); + + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + + Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001); + Assert.assertEquals((long) values[2], agg.getLong(buffer, 0)); + Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001); + } + + @Test + public void testCombine() + { + Assert.assertEquals(3.4d, ((Double) doubleMaxAggFactory.combine(1.2, 3.4)).doubleValue(), 0.0001); + } + + @Test + public void testEqualsAndHashCode() throws Exception + { + DoubleMaxAggregatorFactory one = new DoubleMaxAggregatorFactory("name1", "fieldName1"); + DoubleMaxAggregatorFactory oneMore = new DoubleMaxAggregatorFactory("name1", "fieldName1"); + DoubleMaxAggregatorFactory two = new DoubleMaxAggregatorFactory("name2", "fieldName2"); + + Assert.assertEquals(one.hashCode(), oneMore.hashCode()); + + Assert.assertTrue(one.equals(oneMore)); + Assert.assertFalse(one.equals(two)); + } + + @Test + public void testMaxDeserialization() throws Exception + { + String aggSpecJson = "{\"type\": \"max\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + MaxAggregatorFactory agg = new DefaultObjectMapper().readValue(aggSpecJson , MaxAggregatorFactory.class); + Assert.assertNotNull(agg); + } + + private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg) + { + agg.aggregate(); + selector.increment(); + } + + private void aggregate(TestFloatColumnSelector selector, DoubleMaxBufferAggregator agg, ByteBuffer buff, int position) + { + agg.aggregate(buff, position); + selector.increment(); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java deleted file mode 100644 index b2dc4406957..00000000000 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMaxAggregatorTest.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright 2012 - 2015 Metamarkets Group Inc. - * - * Licensed 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; - -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class DoubleMaxAggregatorTest -{ - private void aggregate(TestFloatColumnSelector selector, DoubleMaxAggregator agg) - { - agg.aggregate(); - selector.increment(); - } - - @Test - public void testAggregate() throws Exception - { - final float[] values = {0.15f, 0.27f, 0.0f, 0.93f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - DoubleMaxAggregator agg = new DoubleMaxAggregator("billy", selector); - - Assert.assertEquals("billy", agg.getName()); - - aggregate(selector, agg); - aggregate(selector, agg); - aggregate(selector, agg); - aggregate(selector, agg); - - Assert.assertEquals(new Float(values[3]).doubleValue(), agg.get()); - } -} diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java new file mode 100644 index 00000000000..95ec2fba15f --- /dev/null +++ b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregationTest.java @@ -0,0 +1,133 @@ +/* +* 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; + +import com.google.common.primitives.Doubles; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.segment.ColumnSelectorFactory; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +/** + */ +public class DoubleMinAggregationTest +{ + private DoubleMinAggregatorFactory doubleMinAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestFloatColumnSelector selector; + + private float[] values = {3.5f, 2.7f, 1.1f, 1.3f}; + + public DoubleMinAggregationTest() throws Exception + { + String aggSpecJson = "{\"type\": \"doubleMin\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + doubleMinAggFactory = new DefaultObjectMapper().readValue(aggSpecJson , DoubleMinAggregatorFactory.class); + } + + @Before + public void setup() + { + selector = new TestFloatColumnSelector(values); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeFloatColumnSelector("nilly")).andReturn(selector); + EasyMock.replay(colSelectorFactory); + } + + @Test + public void testDoubleMinAggregator() + { + DoubleMinAggregator agg = (DoubleMinAggregator) doubleMinAggFactory.factorize(colSelectorFactory); + + Assert.assertEquals("billy", agg.getName()); + + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + aggregate(selector, agg); + + Assert.assertEquals(values[2], ((Double) agg.get()).doubleValue(), 0.0001); + Assert.assertEquals((long)values[2], agg.getLong()); + Assert.assertEquals(values[2], agg.getFloat(), 0.0001); + + agg.reset(); + Assert.assertEquals(Double.POSITIVE_INFINITY, (Double) agg.get(), 0.0001); + } + + @Test + public void testDoubleMinBufferAggregator() + { + DoubleMinBufferAggregator agg = (DoubleMinBufferAggregator) doubleMinAggFactory.factorizeBuffered(colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[Doubles.BYTES]); + agg.init(buffer, 0); + + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + aggregate(selector, agg, buffer, 0); + + Assert.assertEquals(values[2], ((Double) agg.get(buffer, 0)).doubleValue(), 0.0001); + Assert.assertEquals((long) values[2], agg.getLong(buffer, 0)); + Assert.assertEquals(values[2], agg.getFloat(buffer, 0), 0.0001); + } + + @Test + public void testCombine() + { + Assert.assertEquals(1.2d, ((Double) doubleMinAggFactory.combine(1.2, 3.4)).doubleValue(), 0.0001); + } + + @Test + public void testEqualsAndHashCode() throws Exception + { + DoubleMinAggregatorFactory one = new DoubleMinAggregatorFactory("name1", "fieldName1"); + DoubleMinAggregatorFactory oneMore = new DoubleMinAggregatorFactory("name1", "fieldName1"); + DoubleMinAggregatorFactory two = new DoubleMinAggregatorFactory("name2", "fieldName2"); + + Assert.assertEquals(one.hashCode(), oneMore.hashCode()); + + Assert.assertTrue(one.equals(oneMore)); + Assert.assertFalse(one.equals(two)); + } + + @Test + public void testMinDeserialization() throws Exception + { + String aggSpecJson = "{\"type\": \"min\", \"name\": \"billy\", \"fieldName\": \"nilly\"}"; + MinAggregatorFactory agg = new DefaultObjectMapper().readValue(aggSpecJson , MinAggregatorFactory.class); + Assert.assertNotNull(agg); + } + + private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg) + { + agg.aggregate(); + selector.increment(); + } + + private void aggregate(TestFloatColumnSelector selector, DoubleMinBufferAggregator agg, ByteBuffer buff, int position) + { + agg.aggregate(buff, position); + selector.increment(); + } +} diff --git a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java deleted file mode 100644 index 4f74baf8c5d..00000000000 --- a/processing/src/test/java/io/druid/query/aggregation/DoubleMinAggregatorTest.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Druid - a distributed column store. - * Copyright 2012 - 2015 Metamarkets Group Inc. - * - * Licensed 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; - -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class DoubleMinAggregatorTest -{ - private void aggregate(TestFloatColumnSelector selector, DoubleMinAggregator agg) - { - agg.aggregate(); - selector.increment(); - } - - @Test - public void testAggregate() throws Exception - { - final float[] values = {0.15f, 0.27f, 0.0f, 0.93f}; - final TestFloatColumnSelector selector = new TestFloatColumnSelector(values); - DoubleMinAggregator agg = new DoubleMinAggregator("billy", selector); - - Assert.assertEquals("billy", agg.getName()); - - aggregate(selector, agg); - aggregate(selector, agg); - aggregate(selector, agg); - aggregate(selector, agg); - - Assert.assertEquals(new Float(values[2]).doubleValue(), agg.get()); - } -} From 4b907a93c522a8f2aba2a3c859a52b07e62b4b19 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Sun, 10 May 2015 22:17:49 -0500 Subject: [PATCH 5/5] updating the aggregators doc --- docs/content/Aggregations.md | 28 ++++++++++++++++++++++------ 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/docs/content/Aggregations.md b/docs/content/Aggregations.md index 1cbf690020d..71ec7c0f514 100644 --- a/docs/content/Aggregations.md +++ b/docs/content/Aggregations.md @@ -36,20 +36,36 @@ Computes the sum of values as 64-bit floating point value. Similar to `longSum` ### Min / Max aggregators -#### `min` aggregator +#### `doubleMin` aggregator -`min` computes the minimum metric value +`doubleMin` computes the minimum of all metric values and Double.POSITIVE_INFINITY ```json -{ "type" : "min", "name" : , "fieldName" : } +{ "type" : "doubleMin", "name" : , "fieldName" : } ``` -#### `max` aggregator +#### `doubleMax` aggregator -`max` computes the maximum metric value +`doubleMax` computes the maximum of all metric values and Double.NEGATIVE_INFINITY ```json -{ "type" : "max", "name" : , "fieldName" : } +{ "type" : "doubleMax", "name" : , "fieldName" : } +``` + +#### `longMin` aggregator + +`longMin` computes the minimum of all metric values and Long.MAX_VALUE + +```json +{ "type" : "longMin", "name" : , "fieldName" : } +``` + +#### `longMax` aggregator + +`longMax` computes the maximum of all metric values and Long.MIN_VALUE + +```json +{ "type" : "longMax", "name" : , "fieldName" : } ``` ### JavaScript aggregator