From a0049fd4f805b5329582221061319792f40816ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 20 May 2018 15:58:13 +0200 Subject: [PATCH 01/45] Add lastString and firstString aggregators extension --- .../extensions-core/first-last-string.md | 41 +++ docs/content/development/extensions.md | 1 + extensions-contrib/first-last-string/pom.xml | 81 ++++++ .../FirstLastStringDruidModule.java | 59 ++++ .../aggregation/SerializablePairSerde.java | 129 +++++++++ .../first/StringFirstAggregateCombiner.java | 55 ++++ .../first/StringFirstAggregator.java | 100 +++++++ .../first/StringFirstAggregatorFactory.java | 262 +++++++++++++++++ .../first/StringFirstBufferAggregator.java | 125 +++++++++ .../last/SerializablePairSerde.java | 127 +++++++++ .../last/StringLastAggregateCombiner.java | 55 ++++ .../last/StringLastAggregator.java | 100 +++++++ .../last/StringLastAggregatorFactory.java | 263 ++++++++++++++++++ .../last/StringLastBufferAggregator.java | 126 +++++++++ .../io.druid.initialization.DruidModule | 1 + .../first/StringFirstAggregationTest.java | 174 ++++++++++++ .../first/StringFirstAggregatorTest.java | 93 +++++++ .../first/StringFirstTimeseriesQueryTest.java | 130 +++++++++ .../last/StringLastAggregationTest.java | 174 ++++++++++++ .../last/StringLastAggregatorTest.java | 91 ++++++ .../last/StringLastTimeseriesQueryTest.java | 130 +++++++++ .../src/test/resources/sample.data.tsv | 13 + pom.xml | 1 + .../query/aggregation/AggregatorUtil.java | 4 + 24 files changed, 2335 insertions(+) create mode 100644 docs/content/development/extensions-core/first-last-string.md create mode 100644 extensions-contrib/first-last-string/pom.xml create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java create mode 100644 extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java create mode 100644 extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java create mode 100644 extensions-contrib/first-last-string/src/test/resources/sample.data.tsv diff --git a/docs/content/development/extensions-core/first-last-string.md b/docs/content/development/extensions-core/first-last-string.md new file mode 100644 index 000000000000..ab0ba1b00121 --- /dev/null +++ b/docs/content/development/extensions-core/first-last-string.md @@ -0,0 +1,41 @@ +--- +layout: doc_page +--- + +# First/Last String Module + +To use these aggregators, make sure you [include](../../operations/including-extensions.html) the extension in your config file: + +``` +druid.extensions.loadList=["druid-first-last-string"] +``` + +## First String aggregator + +`firstString` computes the metric value with the minimum timestamp or `null` if no row exist + +```json +{ + "type" : "firstString", + "name" : , + "fieldName" : , + "maxStringBytes" : +} +``` + +## Last String aggregator + +```json +{ + "type" : "lastString", + "name" : , + "fieldName" : , + "maxStringBytes" : +} +``` + +`lastString` computes the metric value with the maximum timestamp or `null` if no row exist + + + +Note: The default value of `maxStringBytes` is 1024. diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 652901b2af38..96e31432d680 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -72,6 +72,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c |kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)| |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| |druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)| +|druid-first-last-string|First/Last String Aggregators |[link](../development/extensions-contrib/first-last-string.html)| ## Promoting Community Extension to Core Extension diff --git a/extensions-contrib/first-last-string/pom.xml b/extensions-contrib/first-last-string/pom.xml new file mode 100644 index 000000000000..e7cc7a3f9551 --- /dev/null +++ b/extensions-contrib/first-last-string/pom.xml @@ -0,0 +1,81 @@ + + + + 4.0.0 + + io.druid.extensions.contrib + druid-first-last-string + druid-first-last-string + druid-first-last-string + + + io.druid + druid + 0.13.0-SNAPSHOT + ../../pom.xml + + + + + io.druid + druid-processing + ${project.parent.version} + provided + + + io.druid + druid-sql + ${project.parent.version} + provided + + + + + io.druid + druid-processing + ${project.parent.version} + test + test-jar + + + io.druid + druid-sql + ${project.parent.version} + test-jar + test + + + io.druid + druid-server + ${project.parent.version} + test + test-jar + + + junit + junit + test + + + org.easymock + easymock + test + + + + diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java new file mode 100644 index 000000000000..2f9e63e1ec33 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java @@ -0,0 +1,59 @@ +/* + * 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.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; +import io.druid.query.aggregation.first.StringFirstAggregatorFactory; +import io.druid.query.aggregation.last.StringLastAggregatorFactory; +import io.druid.segment.serde.ComplexMetrics; + +import java.util.List; + +/** + */ +public class FirstLastStringDruidModule implements DruidModule +{ + public static final String STRING_LAST = "stringLast"; + public static final String STRING_FIRST = "stringFirst"; + + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule("FirstLastStringModule").registerSubtypes( + new NamedType(StringLastAggregatorFactory.class, STRING_LAST), + new NamedType(StringFirstAggregatorFactory.class, STRING_FIRST) + ) + ); + } + + @Override + public void configure(Binder binder) + { + if (ComplexMetrics.getSerdeForType("serializablePairLongString") == null) { + ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairSerde()); + } + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java new file mode 100644 index 000000000000..ee4af308c752 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java @@ -0,0 +1,129 @@ +/* + * 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.collections.SerializablePair; +import io.druid.data.input.InputRow; +import io.druid.segment.GenericColumnSerializer; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class SerializablePairSerde extends ComplexMetricSerde +{ + + public SerializablePairSerde() + { + + } + + @Override + public String getTypeName() + { + return "serializablePairLongString"; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return SerializablePair.class; + } + + @Override + public Object extractValue(InputRow inputRow, String metricName) + { + return inputRow.getRaw(metricName); + } + }; + } + + @Override + public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder) + { + final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper()); + columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); + } + + @Override + public ObjectStrategy getObjectStrategy() + { + return new ObjectStrategy() + { + @Override + public int compare(SerializablePair o1, SerializablePair o2) + { + return o1.lhs.equals(o2.lhs) && o1.rhs.equals(o2.rhs) ? 1 : 0; + } + + @Override + public Class getClazz() + { + return SerializablePair.class; + } + + @Override + public SerializablePair fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + + Long lhs = readOnlyBuffer.getLong(); + Integer stringSize = readOnlyBuffer.getInt(); + + byte[] stringBytes = new byte[stringSize]; + readOnlyBuffer.get(stringBytes, 0, stringSize); + + return new SerializablePair<>(lhs, new String(stringBytes, StandardCharsets.UTF_8)); + } + + @Override + public byte[] toBytes(SerializablePair val) + { + String rhsString = (String) val.rhs; + + ByteBuffer bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsString.length()); + bbuf.putLong((Long) val.lhs); + bbuf.putInt(Long.BYTES, rhsString.length()); + bbuf.position(Long.BYTES + Integer.BYTES); + bbuf.put(rhsString.getBytes(StandardCharsets.UTF_8)); + + return bbuf.array(); + } + }; + } + + @Override + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) + { + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java new file mode 100644 index 000000000000..2811847f1bab --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -0,0 +1,55 @@ +/* + * 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.first; + +import io.druid.query.aggregation.ObjectAggregateCombiner; +import io.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +public class StringFirstAggregateCombiner extends ObjectAggregateCombiner +{ + String lastString; + + @Override + public void reset(ColumnValueSelector selector) + { + lastString = (String) selector.getObject(); + } + + @Override + public void fold(ColumnValueSelector selector) + { + lastString = (String) selector.getObject(); + } + + @Nullable + @Override + public String getObject() + { + return lastString; + } + + @Override + public Class classOfObject() + { + return String.class; + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java new file mode 100644 index 000000000000..887e19d1671c --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -0,0 +1,100 @@ +/* + * 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.first; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.BaseLongColumnValueSelector; +import io.druid.segment.BaseObjectColumnValueSelector; + +public class StringFirstAggregator implements Aggregator +{ + + private final BaseObjectColumnValueSelector valueSelector; + private final BaseLongColumnValueSelector timeSelector; + private final Integer maxStringBytes; + + protected long lastTime; + protected String lastValue; + + public StringFirstAggregator( + BaseLongColumnValueSelector timeSelector, + BaseObjectColumnValueSelector valueSelector, + Integer maxStringBytes + ) + { + this.valueSelector = valueSelector; + this.timeSelector = timeSelector; + this.maxStringBytes = maxStringBytes; + + lastTime = Long.MAX_VALUE; + lastValue = null; + } + + @Override + public void aggregate() + { + long time = timeSelector.getLong(); + if (time < lastTime) { + lastTime = time; + Object value = valueSelector.getObject(); + + if (value instanceof String) { + lastValue = (String) valueSelector.getObject(); + + if (lastValue.length() > maxStringBytes) { + lastValue = lastValue.substring(0, maxStringBytes); + } + } else if (value instanceof SerializablePair) { + lastValue = ((SerializablePair) value).rhs; + } + } + } + + @Override + public Object get() + { + return new SerializablePair<>(lastTime, lastValue); + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getFloat()"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getLong()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getDouble()"); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java new file mode 100644 index 000000000000..44ede0781a2c --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -0,0 +1,262 @@ +/* + * 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.first; + +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 io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; +import io.druid.query.aggregation.AggregateCombiner; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseObjectColumnValueSelector; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.column.Column; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class StringFirstAggregatorFactory extends AggregatorFactory +{ + public static final Integer MAX_SIZE_STRING = 1024; + public static final Comparator VALUE_COMPARATOR = (o1, o2) -> ((SerializablePair) o1).rhs.equals(((SerializablePair) o2).rhs) + ? 1 + : 0; + + public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( + ((SerializablePair) o1).lhs, + ((SerializablePair) o2).lhs + ); + + private final String fieldName; + private final String name; + private final Integer maxStringBytes; + + @JsonCreator + public StringFirstAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("maxStringBytes") Integer maxStringBytes + ) + { + 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; + this.maxStringBytes = maxStringBytes == null ? MAX_SIZE_STRING : maxStringBytes; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new StringFirstAggregator( + metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), + metricFactory.makeColumnValueSelector(fieldName), + maxStringBytes + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new StringFirstBufferAggregator( + metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), + metricFactory.makeColumnValueSelector(fieldName), + maxStringBytes + ); + } + + @Override + public Comparator getComparator() + { + return VALUE_COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new StringFirstAggregateCombiner(); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new StringFirstAggregatorFactory(name, name, maxStringBytes) + { + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringFirstAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate() + { + SerializablePair pair = (SerializablePair) selector.getObject(); + if (pair.lhs < lastTime) { + lastTime = pair.lhs; + lastValue = pair.rhs; + } + } + }; + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringFirstBufferAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + SerializablePair pair = (SerializablePair) selector.getObject(); + long lastTime = mutationBuffer.getLong(position); + if (pair.lhs < lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + }; + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); + } + + @Override + public Object deserialize(Object object) + { + Map map = (Map) object; + return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); + } + + @Override + public Object finalizeComputation(Object object) + { + return ((SerializablePair) object).rhs; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + public List requiredFields() + { + return Arrays.asList(Column.TIME_COLUMN_NAME, fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length) + .put(AggregatorUtil.STRING_FIRST_CACHE_TYPE_ID) + .put(fieldNameBytes) + .array(); + } + + @Override + public String getTypeName() + { + return "serializablePairLongString"; + } + + @Override + public int getMaxIntermediateSize() + { + return Long.BYTES + Integer.BYTES + maxStringBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StringFirstAggregatorFactory that = (StringFirstAggregatorFactory) o; + + return fieldName.equals(that.fieldName) && name.equals(that.name); + } + + @Override + public int hashCode() + { + return Objects.hash(name, fieldName); + } + + @Override + public String toString() + { + return "StringFirstAggregatorFactory{" + + "name='" + name + '\'' + + ", fieldName='" + fieldName + '\'' + + ", maxStringBytes=" + maxStringBytes + '\'' + + '}'; + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java new file mode 100644 index 000000000000..b78da5547ccd --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.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.first; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseLongColumnValueSelector; +import io.druid.segment.BaseObjectColumnValueSelector; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class StringFirstBufferAggregator implements BufferAggregator +{ + private final BaseLongColumnValueSelector timeSelector; + private final BaseObjectColumnValueSelector valueSelector; + private final Integer maxStringBytes; + + public StringFirstBufferAggregator( + BaseLongColumnValueSelector timeSelector, + BaseObjectColumnValueSelector valueSelector, + Integer maxStringBytes + ) + { + this.timeSelector = timeSelector; + this.valueSelector = valueSelector; + this.maxStringBytes = maxStringBytes; + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + mutationBuffer.putLong(position, Long.MAX_VALUE); + mutationBuffer.putInt(position + Long.BYTES, 0); + for (int i = 0; i < maxStringBytes - 1; i++) { + mutationBuffer.putChar(position + Long.BYTES + Integer.BYTES + i, '\0'); + } + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + long time = timeSelector.getLong(); + long lastTime = mutationBuffer.getLong(position); + if (time < lastTime) { + byte[] valueBytes = ((String) valueSelector.getObject()).getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } + } + + @Override + public Object get(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + Long timeValue = mutationBuffer.getLong(position); + Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); + + byte[] valueBytes = new byte[stringSizeBytes]; + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.get(valueBytes, 0, stringSizeBytes); + return new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getFloat()"); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getLong()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getDouble()"); + } + + @Override + public void close() + { + // no resources to cleanup + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java new file mode 100644 index 000000000000..347037ff5177 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java @@ -0,0 +1,127 @@ +/* + * 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.last; + +import io.druid.collections.SerializablePair; +import io.druid.data.input.InputRow; +import io.druid.segment.GenericColumnSerializer; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.ObjectStrategy; +import io.druid.segment.serde.ComplexColumnPartSupplier; +import io.druid.segment.serde.ComplexMetricExtractor; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; +import io.druid.segment.writeout.SegmentWriteOutMedium; + +import java.nio.ByteBuffer; + +public class SerializablePairSerde extends ComplexMetricSerde +{ + + public SerializablePairSerde() + { + + } + + @Override + public String getTypeName() + { + return "serializablePairLongString"; + } + + @Override + public ComplexMetricExtractor getExtractor() + { + return new ComplexMetricExtractor() + { + @Override + public Class extractedClass() + { + return SerializablePair.class; + } + + @Override + public Object extractValue(InputRow inputRow, String metricName) + { + return inputRow.getRaw(metricName); + } + }; + } + + @Override + public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder) + { + final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper()); + columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); + } + + @Override + public ObjectStrategy getObjectStrategy() + { + return new ObjectStrategy() + { + @Override + public int compare(SerializablePair o1, SerializablePair o2) + { + return o1.lhs.equals(o2.lhs) && o1.rhs.equals(o2.rhs) ? 1 : 0; + } + + @Override + public Class getClazz() + { + return SerializablePair.class; + } + + @Override + public SerializablePair fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); + + Long lhs = readOnlyBuffer.getLong(); + Integer stringSize = readOnlyBuffer.getInt(); + + byte[] stringBytes = new byte[stringSize]; + readOnlyBuffer.get(stringBytes, 0, stringSize); + + return new SerializablePair<>(lhs, new String(stringBytes)); + } + + @Override + public byte[] toBytes(SerializablePair val) + { + String rhsString = (String) val.rhs; + + ByteBuffer bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsString.length()); + bbuf.putLong((Long) val.lhs); + bbuf.putInt(rhsString.length()); + bbuf.put(rhsString.getBytes()); + + return bbuf.array(); + } + }; + } + + @Override + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) + { + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java new file mode 100644 index 000000000000..19cc6dd6d3ec --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java @@ -0,0 +1,55 @@ +/* + * 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.last; + +import io.druid.query.aggregation.ObjectAggregateCombiner; +import io.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +public class StringLastAggregateCombiner extends ObjectAggregateCombiner +{ + String lastString; + + @Override + public void reset(ColumnValueSelector selector) + { + lastString = (String) selector.getObject(); + } + + @Override + public void fold(ColumnValueSelector selector) + { + lastString = (String) selector.getObject(); + } + + @Nullable + @Override + public String getObject() + { + return lastString; + } + + @Override + public Class classOfObject() + { + return String.class; + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java new file mode 100644 index 000000000000..61bc387333f4 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -0,0 +1,100 @@ +/* + * 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.last; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.Aggregator; +import io.druid.segment.BaseLongColumnValueSelector; +import io.druid.segment.BaseObjectColumnValueSelector; + +public class StringLastAggregator implements Aggregator +{ + + private final BaseObjectColumnValueSelector valueSelector; + private final BaseLongColumnValueSelector timeSelector; + private final Integer maxStringBytes; + + protected long lastTime; + protected String lastValue; + + public StringLastAggregator( + BaseLongColumnValueSelector timeSelector, + BaseObjectColumnValueSelector valueSelector, + Integer maxStringBytes + ) + { + this.valueSelector = valueSelector; + this.timeSelector = timeSelector; + this.maxStringBytes = maxStringBytes; + + lastTime = Long.MIN_VALUE; + lastValue = null; + } + + @Override + public void aggregate() + { + long time = timeSelector.getLong(); + if (time >= lastTime) { + lastTime = time; + Object value = valueSelector.getObject(); + + if (value instanceof String) { + lastValue = (String) valueSelector.getObject(); + + if (lastValue.length() > maxStringBytes) { + lastValue = lastValue.substring(0, maxStringBytes); + } + } else if (value instanceof SerializablePair) { + lastValue = ((SerializablePair) value).rhs; + } + } + } + + @Override + public Object get() + { + return new SerializablePair<>(lastTime, lastValue); + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getFloat()"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getLong()"); + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getDouble()"); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java new file mode 100644 index 000000000000..e1ac198aa22e --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -0,0 +1,263 @@ +/* + * 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.last; + +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 io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; +import io.druid.query.aggregation.AggregateCombiner; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.AggregatorUtil; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseObjectColumnValueSelector; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.column.Column; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class StringLastAggregatorFactory extends AggregatorFactory +{ + public static final Integer MAX_SIZE_STRING = 1024; + public static final Comparator VALUE_COMPARATOR = (o1, o2) -> ((SerializablePair) o1).rhs.equals(((SerializablePair) o2).rhs) + ? 1 + : 0; + + public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( + ((SerializablePair) o1).lhs, + ((SerializablePair) o2).lhs + ); + + private final String fieldName; + private final String name; + private final Integer maxStringBytes; + + @JsonCreator + public StringLastAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("maxStringBytes") Integer maxStringBytes + ) + { + 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; + this.maxStringBytes = maxStringBytes == null ? MAX_SIZE_STRING : maxStringBytes; + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + return new StringLastAggregator( + metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), + metricFactory.makeColumnValueSelector(fieldName), + maxStringBytes + ); + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + return new StringLastBufferAggregator( + metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), + metricFactory.makeColumnValueSelector(fieldName), + maxStringBytes + ); + } + + @Override + public Comparator getComparator() + { + return VALUE_COMPARATOR; + } + + @Override + public Object combine(Object lhs, Object rhs) + { + return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new StringLastAggregateCombiner(); + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new StringLastAggregatorFactory(name, name, maxStringBytes) + { + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringLastAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate() + { + SerializablePair pair = (SerializablePair) selector.getObject(); + if (pair.lhs >= lastTime) { + lastTime = pair.lhs; + lastValue = pair.rhs; + } + } + }; + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringLastBufferAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + SerializablePair pair = (SerializablePair) selector.getObject(); + long lastTime = mutationBuffer.getLong(position); + if (pair.lhs >= lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + }; + } + + @Override + public List getRequiredColumns() + { + return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); + } + + @Override + public Object deserialize(Object object) + { + Map map = (Map) object; + return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); + } + + @Override + public Object finalizeComputation(Object object) + { + return ((SerializablePair) object).rhs; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @Override + public List requiredFields() + { + return Arrays.asList(Column.TIME_COLUMN_NAME, fieldName); + } + + @Override + public byte[] getCacheKey() + { + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + return ByteBuffer.allocate(1 + fieldNameBytes.length) + .put(AggregatorUtil.STRING_LAST_CACHE_TYPE_ID) + .put(fieldNameBytes) + .array(); + } + + @Override + public String getTypeName() + { + return "serializablePairLongString"; + } + + @Override + public int getMaxIntermediateSize() + { + return Long.BYTES + Integer.BYTES + maxStringBytes; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + StringLastAggregatorFactory that = (StringLastAggregatorFactory) o; + + return fieldName.equals(that.fieldName) && name.equals(that.name); + } + + @Override + public int hashCode() + { + return Objects.hash(name, fieldName); + } + + @Override + public String toString() + { + return "StringFirstAggregatorFactory{" + + "name='" + name + '\'' + + ", fieldName='" + fieldName + '\'' + + ", maxStringBytes=" + maxStringBytes + '\'' + + '}'; + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java new file mode 100644 index 000000000000..181ad0e18cd2 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -0,0 +1,126 @@ +/* + * 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.last; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseLongColumnValueSelector; +import io.druid.segment.BaseObjectColumnValueSelector; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class StringLastBufferAggregator implements BufferAggregator +{ + private final BaseLongColumnValueSelector timeSelector; + private final BaseObjectColumnValueSelector valueSelector; + private final Integer maxStringBytes; + + public StringLastBufferAggregator( + BaseLongColumnValueSelector timeSelector, + BaseObjectColumnValueSelector valueSelector, + Integer maxStringBytes + ) + { + this.timeSelector = timeSelector; + this.valueSelector = valueSelector; + this.maxStringBytes = maxStringBytes; + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + mutationBuffer.putLong(position, Long.MIN_VALUE); + mutationBuffer.putInt(position + Long.BYTES, 0); + for (int i = 0; i < maxStringBytes - 1; i++) { + mutationBuffer.putChar(position + Long.BYTES + Integer.BYTES + i, '\0'); + } + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + long time = timeSelector.getLong(); + long lastTime = mutationBuffer.getLong(position); + if (time >= lastTime) { + byte[] valueBytes = ((String) valueSelector.getObject()).getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + + } + } + + @Override + public Object get(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + Long timeValue = mutationBuffer.getLong(position); + Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); + + byte[] valueBytes = new byte[stringSizeBytes]; + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.get(valueBytes, 0, stringSizeBytes); + return new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getFloat()"); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getLong()"); + } + + @Override + public double getDouble(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException("StringFirstAggregator does not support getDouble()"); + } + + @Override + public void close() + { + // no resources to cleanup + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("timeSelector", timeSelector); + inspector.visit("valueSelector", valueSelector); + } +} diff --git a/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100644 index 000000000000..981e7aee02fc --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.query.aggregation.FirstLastStringDruidModule \ No newline at end of file diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java new file mode 100644 index 000000000000..857aa0511005 --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -0,0 +1,174 @@ +/* + * 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.first; + +import io.druid.collections.SerializablePair; +import io.druid.java.util.common.Pair; +import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.TestLongColumnSelector; +import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.column.Column; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +public class StringFirstAggregationTest +{ + private final Integer MAX_STRING_SIZE = 1024; + private StringFirstAggregatorFactory stringLastAggFactory; + private StringFirstAggregatorFactory combiningAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestLongColumnSelector timeSelector; + private TestObjectColumnSelector valueSelector; + private TestObjectColumnSelector objectSelector; + + private String[] strings = {"1111", "2222", "3333", "4444"}; + private long[] times = {8224, 6879, 2436, 7888}; + private SerializablePair[] pairs = { + new SerializablePair<>(52782L, "AAAA"), + new SerializablePair<>(65492L, "BBBB"), + new SerializablePair<>(69134L, "CCCC"), + new SerializablePair<>(11111L, "DDDD") + }; + + @Before + public void setup() + { + stringLastAggFactory = new StringFirstAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); + combiningAggFactory = (StringFirstAggregatorFactory) stringLastAggFactory.getCombiningFactory(); + timeSelector = new TestLongColumnSelector(times); + valueSelector = new TestObjectColumnSelector<>(strings); + objectSelector = new TestObjectColumnSelector<>(pairs); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME)).andReturn(timeSelector); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); + EasyMock.replay(colSelectorFactory); + + FirstLastStringDruidModule module = new FirstLastStringDruidModule(); + module.configure(null); + } + + @Test + public void testStringLastAggregator() + { + StringFirstAggregator agg = (StringFirstAggregator) stringLastAggFactory.factorize(colSelectorFactory); + + aggregate(agg); + aggregate(agg); + aggregate(agg); + aggregate(agg); + + Pair result = (Pair) agg.get(); + + Assert.assertEquals(strings[2], result.rhs); + } + + @Test + public void testStringLastBufferAggregator() + { + StringFirstBufferAggregator agg = (StringFirstBufferAggregator) stringLastAggFactory.factorizeBuffered( + colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[stringLastAggFactory.getMaxIntermediateSize()]); + agg.init(buffer, 0); + + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + + Pair result = (Pair) agg.get(buffer, 0); + + Assert.assertEquals(strings[2], result.rhs); + } + + @Test + public void testCombine() + { + SerializablePair pair1 = new SerializablePair<>(1467225000L, "AAAA"); + SerializablePair pair2 = new SerializablePair<>(1467240000L, "BBBB"); + Assert.assertEquals(pair2, stringLastAggFactory.combine(pair1, pair2)); + } + + @Test + public void testStringLastCombiningAggregator() + { + StringFirstAggregator agg = (StringFirstAggregator) combiningAggFactory.factorize(colSelectorFactory); + + aggregate(agg); + aggregate(agg); + aggregate(agg); + aggregate(agg); + + Pair result = (Pair) agg.get(); + Pair expected = (Pair) pairs[3]; + + Assert.assertEquals(expected.lhs, result.lhs); + Assert.assertEquals(expected.rhs, result.rhs); + } + + @Test + public void testStringLastCombiningBufferAggregator() + { + StringFirstBufferAggregator agg = (StringFirstBufferAggregator) combiningAggFactory.factorizeBuffered( + colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[stringLastAggFactory.getMaxIntermediateSize()]); + agg.init(buffer, 0); + + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + + Pair result = (Pair) agg.get(buffer, 0); + Pair expected = (Pair) pairs[3]; + + Assert.assertEquals(expected.lhs, result.lhs); + Assert.assertEquals(expected.rhs, result.rhs); + } + + private void aggregate( + StringFirstAggregator agg + ) + { + agg.aggregate(); + timeSelector.increment(); + valueSelector.increment(); + objectSelector.increment(); + } + + private void aggregate( + StringFirstBufferAggregator agg, + ByteBuffer buff, + int position + ) + { + agg.aggregate(buff, position); + timeSelector.increment(); + valueSelector.increment(); + objectSelector.increment(); + } +} diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java new file mode 100644 index 000000000000..bc2c81dc3c3f --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java @@ -0,0 +1,93 @@ +/* + * 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.first; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.TestLongColumnSelector; +import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.query.aggregation.last.StringLastAggregatorFactory; +import io.druid.query.aggregation.last.StringLastBufferAggregator; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class StringFirstAggregatorTest +{ + private void aggregateBuffer( + TestLongColumnSelector timeSelector, + TestObjectColumnSelector valueSelector, + BufferAggregator agg, + ByteBuffer buf, + int position + ) + { + agg.aggregate(buf, position); + timeSelector.increment(); + valueSelector.increment(); + } + + @Test + public void testBufferAggregate() throws Exception + { + + final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; + final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePair sp = ((SerializablePair) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", "DDDD", sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(1526725900L), new Long(sp.lhs)); + + } + +} diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java new file mode 100644 index 000000000000..446d0c34c092 --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -0,0 +1,130 @@ +/* + * 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.first; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.collections.SerializablePair; +import io.druid.data.input.MapBasedInputRow; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.Druids; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.Result; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.TestHelper; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.joda.time.DateTime; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class StringFirstTimeseriesQueryTest +{ + + @Before + public void setup() + { + FirstLastStringDruidModule module = new FirstLastStringDruidModule(); + module.configure(null); + } + + @Test + public void testTopNWithDistinctCountAgg() throws Exception + { + TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); + + String visitor_id = "visitor_id"; + String client_type = "client_type"; + + IncrementalIndex index = new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withQueryGranularity(Granularities.SECOND) + .withMetrics(new CountAggregatorFactory("cnt")) + .withMetrics(new StringFirstAggregatorFactory( + "last_client_type", "client_type", 1024) + ) + .build() + ) + .setMaxRowCount(1000) + .buildOnheap(); + + + DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z"); + long timestamp = time.getMillis(); + + DateTime time1 = DateTimes.of("2016-03-04T01:00:00.000Z"); + long timestamp1 = time1.getMillis(); + index.add( + new MapBasedInputRow( + timestamp, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "0", client_type, "iphone") + ) + ); + index.add( + new MapBasedInputRow( + timestamp, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "1", client_type, "iphone") + ) + ); + index.add( + new MapBasedInputRow( + timestamp1, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "0", client_type, "android") + ) + ); + + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + new StringFirstAggregatorFactory("last_client_type", client_type, 1024) + ) + ) + .build(); + + final Iterable> results = + engine.process(query, new IncrementalIndexStorageAdapter(index)).toList(); + + List> expectedResults = Collections.singletonList( + new Result<>( + time, + new TimeseriesResultValue( + ImmutableMap.of("last_client_type", new SerializablePair<>(timestamp, "iphone")) + ) + ) + ); + TestHelper.assertExpectedResults(expectedResults, results); + } +} diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java new file mode 100644 index 000000000000..c5bc7c7c53a9 --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -0,0 +1,174 @@ +/* + * 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.last; + +import io.druid.collections.SerializablePair; +import io.druid.java.util.common.Pair; +import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.TestLongColumnSelector; +import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.segment.ColumnSelectorFactory; +import io.druid.segment.column.Column; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +public class StringLastAggregationTest +{ + private final Integer MAX_STRING_SIZE = 1024; + private StringLastAggregatorFactory stringLastAggFactory; + private StringLastAggregatorFactory combiningAggFactory; + private ColumnSelectorFactory colSelectorFactory; + private TestLongColumnSelector timeSelector; + private TestObjectColumnSelector valueSelector; + private TestObjectColumnSelector objectSelector; + + private String[] strings = {"1111", "2222", "3333", "4444"}; + private long[] times = {8224, 6879, 2436, 7888}; + private SerializablePair[] pairs = { + new SerializablePair<>(52782L, "AAAA"), + new SerializablePair<>(65492L, "BBBB"), + new SerializablePair<>(69134L, "CCCC"), + new SerializablePair<>(11111L, "DDDD") + }; + + @Before + public void setup() + { + stringLastAggFactory = new StringLastAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); + combiningAggFactory = (StringLastAggregatorFactory) stringLastAggFactory.getCombiningFactory(); + timeSelector = new TestLongColumnSelector(times); + valueSelector = new TestObjectColumnSelector<>(strings); + objectSelector = new TestObjectColumnSelector<>(pairs); + colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME)).andReturn(timeSelector); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); + EasyMock.replay(colSelectorFactory); + + FirstLastStringDruidModule module = new FirstLastStringDruidModule(); + module.configure(null); + } + + @Test + public void testStringLastAggregator() + { + StringLastAggregator agg = (StringLastAggregator) stringLastAggFactory.factorize(colSelectorFactory); + + aggregate(agg); + aggregate(agg); + aggregate(agg); + aggregate(agg); + + Pair result = (Pair) agg.get(); + + Assert.assertEquals(strings[0], result.rhs); + } + + @Test + public void testStringLastBufferAggregator() + { + StringLastBufferAggregator agg = (StringLastBufferAggregator) stringLastAggFactory.factorizeBuffered( + colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[stringLastAggFactory.getMaxIntermediateSize()]); + agg.init(buffer, 0); + + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + + Pair result = (Pair) agg.get(buffer, 0); + + Assert.assertEquals(strings[0], result.rhs); + } + + @Test + public void testCombine() + { + SerializablePair pair1 = new SerializablePair<>(1467225000L, "AAAA"); + SerializablePair pair2 = new SerializablePair<>(1467240000L, "BBBB"); + Assert.assertEquals(pair2, stringLastAggFactory.combine(pair1, pair2)); + } + + @Test + public void testStringLastCombiningAggregator() + { + StringLastAggregator agg = (StringLastAggregator) combiningAggFactory.factorize(colSelectorFactory); + + aggregate(agg); + aggregate(agg); + aggregate(agg); + aggregate(agg); + + Pair result = (Pair) agg.get(); + Pair expected = (Pair) pairs[2]; + + Assert.assertEquals(expected.lhs, result.lhs); + Assert.assertEquals(expected.rhs, result.rhs); + } + + @Test + public void testStringLastCombiningBufferAggregator() + { + StringLastBufferAggregator agg = (StringLastBufferAggregator) combiningAggFactory.factorizeBuffered( + colSelectorFactory); + + ByteBuffer buffer = ByteBuffer.wrap(new byte[stringLastAggFactory.getMaxIntermediateSize()]); + agg.init(buffer, 0); + + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + aggregate(agg, buffer, 0); + + Pair result = (Pair) agg.get(buffer, 0); + Pair expected = (Pair) pairs[2]; + + Assert.assertEquals(expected.lhs, result.lhs); + Assert.assertEquals(expected.rhs, result.rhs); + } + + private void aggregate( + StringLastAggregator agg + ) + { + agg.aggregate(); + timeSelector.increment(); + valueSelector.increment(); + objectSelector.increment(); + } + + private void aggregate( + StringLastBufferAggregator agg, + ByteBuffer buff, + int position + ) + { + agg.aggregate(buff, position); + timeSelector.increment(); + valueSelector.increment(); + objectSelector.increment(); + } +} diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java new file mode 100644 index 000000000000..aea6a53784e5 --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java @@ -0,0 +1,91 @@ +/* + * 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.last; + +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.TestLongColumnSelector; +import io.druid.query.aggregation.TestObjectColumnSelector; +import org.junit.Assert; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +public class StringLastAggregatorTest +{ + private void aggregateBuffer( + TestLongColumnSelector timeSelector, + TestObjectColumnSelector valueSelector, + BufferAggregator agg, + ByteBuffer buf, + int position + ) + { + agg.aggregate(buf, position); + timeSelector.increment(); + valueSelector.increment(); + } + + @Test + public void testBufferAggregate() throws Exception + { + + final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; + final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePair sp = ((SerializablePair) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", "DDDD", sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(1526725900L), new Long(sp.lhs)); + + } + +} diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java new file mode 100644 index 000000000000..11e88256fe41 --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -0,0 +1,130 @@ +/* + * 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.last; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.druid.collections.SerializablePair; +import io.druid.data.input.MapBasedInputRow; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.query.Druids; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.Result; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.TestHelper; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.joda.time.DateTime; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.List; + +public class StringLastTimeseriesQueryTest +{ + + @Before + public void setup() + { + FirstLastStringDruidModule module = new FirstLastStringDruidModule(); + module.configure(null); + } + + @Test + public void testTopNWithDistinctCountAgg() throws Exception + { + TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); + + String visitor_id = "visitor_id"; + String client_type = "client_type"; + + IncrementalIndex index = new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withQueryGranularity(Granularities.SECOND) + .withMetrics(new CountAggregatorFactory("cnt")) + .withMetrics(new StringLastAggregatorFactory( + "last_client_type", "client_type", 1024) + ) + .build() + ) + .setMaxRowCount(1000) + .buildOnheap(); + + + DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z"); + long timestamp = time.getMillis(); + + DateTime time1 = DateTimes.of("2016-03-04T01:00:00.000Z"); + long timestamp1 = time1.getMillis(); + index.add( + new MapBasedInputRow( + timestamp, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "0", client_type, "iphone") + ) + ); + index.add( + new MapBasedInputRow( + timestamp, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "1", client_type, "iphone") + ) + ); + index.add( + new MapBasedInputRow( + timestamp1, + Lists.newArrayList(visitor_id, client_type), + ImmutableMap.of(visitor_id, "0", client_type, "android") + ) + ); + + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.dataSource) + .granularity(QueryRunnerTestHelper.allGran) + .intervals(QueryRunnerTestHelper.fullOnInterval) + .aggregators( + Lists.newArrayList( + new StringLastAggregatorFactory("last_client_type", client_type, 1024) + ) + ) + .build(); + + final Iterable> results = + engine.process(query, new IncrementalIndexStorageAdapter(index)).toList(); + + List> expectedResults = Collections.singletonList( + new Result<>( + time, + new TimeseriesResultValue( + ImmutableMap.of("last_client_type", new SerializablePair<>(timestamp1, "android")) + ) + ) + ); + TestHelper.assertExpectedResults(expectedResults, results); + } +} diff --git a/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv b/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv new file mode 100644 index 000000000000..674d86cefe9f --- /dev/null +++ b/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv @@ -0,0 +1,13 @@ +2011-04-15T00:00:00.000Z spot automotive preferred apreferred 106.793700 +2011-04-15T00:00:00.000Z spot business preferred bpreferred 94.469747 +2011-04-15T00:00:00.000Z spot entertainment preferred epreferred 135.109191 +2011-04-15T00:00:00.000Z spot health preferred hpreferred 99.596909 +2011-04-15T00:00:00.000Z spot mezzanine preferred mpreferred 92.782760 +2011-04-15T00:00:00.000Z spot news preferred npreferred +2011-04-15T00:00:00.000Z spot premium preferred ppreferred +2011-04-15T00:00:00.000Z spot technology preferred tpreferred +2011-04-15T00:00:00.000Z spot travel preferred tpreferred +2011-04-15T00:00:00.000Z total_market mezzanine preferred mpreferred +2011-04-15T00:00:00.000Z total_market premium preferred ppreferred +2011-04-15T00:00:00.000Z upfront mezzanine preferred mpreferred +2011-04-15T00:00:00.000Z upfront premium preferred ppreferred diff --git a/pom.xml b/pom.xml index 4dc14357f053..98400d4dc700 100644 --- a/pom.xml +++ b/pom.xml @@ -146,6 +146,7 @@ extensions-contrib/kafka-emitter extensions-contrib/redis-cache extensions-contrib/opentsdb-emitter + extensions-contrib/first-last-string distribution diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java index 833eed385602..3a0211f0b3e3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregatorUtil.java @@ -94,6 +94,10 @@ public class AggregatorUtil public static final byte ARRAY_OF_DOUBLES_SKETCH_T_TEST_CACHE_TYPE_ID = 0x29; public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x2A; + // StringFirst, StringLast aggregator + public static final byte STRING_FIRST_CACHE_TYPE_ID = 0x2B; + public static final byte STRING_LAST_CACHE_TYPE_ID = 0x2C; + /** * returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg * From 09338b95b4dc2f97cfa97da63dc56dfc1c7d001d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 20 May 2018 16:21:49 +0200 Subject: [PATCH 02/45] Remove duplicated class --- .../last/SerializablePairSerde.java | 127 ------------------ 1 file changed, 127 deletions(-) delete mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java deleted file mode 100644 index 347037ff5177..000000000000 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/SerializablePairSerde.java +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.last; - -import io.druid.collections.SerializablePair; -import io.druid.data.input.InputRow; -import io.druid.segment.GenericColumnSerializer; -import io.druid.segment.column.ColumnBuilder; -import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.ObjectStrategy; -import io.druid.segment.serde.ComplexColumnPartSupplier; -import io.druid.segment.serde.ComplexMetricExtractor; -import io.druid.segment.serde.ComplexMetricSerde; -import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; -import io.druid.segment.writeout.SegmentWriteOutMedium; - -import java.nio.ByteBuffer; - -public class SerializablePairSerde extends ComplexMetricSerde -{ - - public SerializablePairSerde() - { - - } - - @Override - public String getTypeName() - { - return "serializablePairLongString"; - } - - @Override - public ComplexMetricExtractor getExtractor() - { - return new ComplexMetricExtractor() - { - @Override - public Class extractedClass() - { - return SerializablePair.class; - } - - @Override - public Object extractValue(InputRow inputRow, String metricName) - { - return inputRow.getRaw(metricName); - } - }; - } - - @Override - public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder) - { - final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper()); - columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); - } - - @Override - public ObjectStrategy getObjectStrategy() - { - return new ObjectStrategy() - { - @Override - public int compare(SerializablePair o1, SerializablePair o2) - { - return o1.lhs.equals(o2.lhs) && o1.rhs.equals(o2.rhs) ? 1 : 0; - } - - @Override - public Class getClazz() - { - return SerializablePair.class; - } - - @Override - public SerializablePair fromByteBuffer(ByteBuffer buffer, int numBytes) - { - final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); - - Long lhs = readOnlyBuffer.getLong(); - Integer stringSize = readOnlyBuffer.getInt(); - - byte[] stringBytes = new byte[stringSize]; - readOnlyBuffer.get(stringBytes, 0, stringSize); - - return new SerializablePair<>(lhs, new String(stringBytes)); - } - - @Override - public byte[] toBytes(SerializablePair val) - { - String rhsString = (String) val.rhs; - - ByteBuffer bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsString.length()); - bbuf.putLong((Long) val.lhs); - bbuf.putInt(rhsString.length()); - bbuf.put(rhsString.getBytes()); - - return bbuf.array(); - } - }; - } - - @Override - public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) - { - return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); - } -} From 3658b0cd71efb79b26f4d357b47666937a94df81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 20 May 2018 17:09:48 +0200 Subject: [PATCH 03/45] Move first-last-string doc page to extensions-contrib --- .../{extensions-core => extensions-contrib}/first-last-string.md | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/content/development/{extensions-core => extensions-contrib}/first-last-string.md (100%) diff --git a/docs/content/development/extensions-core/first-last-string.md b/docs/content/development/extensions-contrib/first-last-string.md similarity index 100% rename from docs/content/development/extensions-core/first-last-string.md rename to docs/content/development/extensions-contrib/first-last-string.md From 5975952123b8cb40ff51eb2bfd0c1783414cb8b3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 28 May 2018 11:57:08 +0200 Subject: [PATCH 04/45] Fix ObjectStrategy compare method --- .../query/aggregation/SerializablePairSerde.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java index ee4af308c752..05af9150b6b3 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java @@ -82,7 +82,21 @@ public ObjectStrategy getObjectStrategy() @Override public int compare(SerializablePair o1, SerializablePair o2) { - return o1.lhs.equals(o2.lhs) && o1.rhs.equals(o2.rhs) ? 1 : 0; + Integer comparation = 0; + + if ((Long) o1.lhs > (Long) o2.lhs) { + comparation = 1; + } else if ((Long) o1.lhs < (Long) o2.lhs) { + comparation = -1; + } + + if (comparation == 0 && o1.rhs.equals(o2.rhs)) { + comparation = 0; + } else if (comparation == 0) { + comparation = -1; + } + + return comparation; } @Override From adc773be15998070992996d0196ce072da55950a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Tue, 29 May 2018 14:01:37 +0200 Subject: [PATCH 05/45] Fix doc bad aggregatos type name --- .../development/extensions-contrib/first-last-string.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content/development/extensions-contrib/first-last-string.md b/docs/content/development/extensions-contrib/first-last-string.md index ab0ba1b00121..e5a83ae86748 100644 --- a/docs/content/development/extensions-contrib/first-last-string.md +++ b/docs/content/development/extensions-contrib/first-last-string.md @@ -12,11 +12,11 @@ druid.extensions.loadList=["druid-first-last-string"] ## First String aggregator -`firstString` computes the metric value with the minimum timestamp or `null` if no row exist +`stringFirst` computes the metric value with the minimum timestamp or `null` if no row exist ```json { - "type" : "firstString", + "type" : "stringFirst", "name" : , "fieldName" : , "maxStringBytes" : @@ -27,14 +27,14 @@ druid.extensions.loadList=["druid-first-last-string"] ```json { - "type" : "lastString", + "type" : "stringLast", "name" : , "fieldName" : , "maxStringBytes" : } ``` -`lastString` computes the metric value with the maximum timestamp or `null` if no row exist +`stringLast` computes the metric value with the maximum timestamp or `null` if no row exist From c2f367245bb2f559cc5670f40966b67a4381896e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Thu, 31 May 2018 14:20:13 +0200 Subject: [PATCH 06/45] Create FoldingAggregatorFactory classes to fix SegmentMetadataQuery --- .../FirstLastStringDruidModule.java | 11 ++- .../first/StringFirstAggregator.java | 22 ++--- .../first/StringFirstAggregatorFactory.java | 70 ++------------ .../StringFirstFoldingAggregatorFactory.java | 96 +++++++++++++++++++ .../last/StringLastAggregator.java | 2 +- .../last/StringLastAggregatorFactory.java | 64 ++----------- .../StringLastFoldingAggregatorFactory.java | 95 ++++++++++++++++++ 7 files changed, 221 insertions(+), 139 deletions(-) create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java index 2f9e63e1ec33..ee84133f6eba 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java @@ -20,13 +20,14 @@ package io.druid.query.aggregation; import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import io.druid.initialization.DruidModule; import io.druid.query.aggregation.first.StringFirstAggregatorFactory; +import io.druid.query.aggregation.first.StringFirstFoldingAggregatorFactory; import io.druid.query.aggregation.last.StringLastAggregatorFactory; +import io.druid.query.aggregation.last.StringLastFoldingAggregatorFactory; import io.druid.segment.serde.ComplexMetrics; import java.util.List; @@ -35,16 +36,16 @@ */ public class FirstLastStringDruidModule implements DruidModule { - public static final String STRING_LAST = "stringLast"; - public static final String STRING_FIRST = "stringFirst"; @Override public List getJacksonModules() { return ImmutableList.of( new SimpleModule("FirstLastStringModule").registerSubtypes( - new NamedType(StringLastAggregatorFactory.class, STRING_LAST), - new NamedType(StringFirstAggregatorFactory.class, STRING_FIRST) + StringLastAggregatorFactory.class, + StringLastFoldingAggregatorFactory.class, + StringFirstAggregatorFactory.class, + StringFirstFoldingAggregatorFactory.class ) ); } diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index 887e19d1671c..6a4fd40931f0 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -31,8 +31,8 @@ public class StringFirstAggregator implements Aggregator private final BaseLongColumnValueSelector timeSelector; private final Integer maxStringBytes; - protected long lastTime; - protected String lastValue; + protected long firstTime; + protected String firstValue; public StringFirstAggregator( BaseLongColumnValueSelector timeSelector, @@ -44,26 +44,26 @@ public StringFirstAggregator( this.timeSelector = timeSelector; this.maxStringBytes = maxStringBytes; - lastTime = Long.MAX_VALUE; - lastValue = null; + firstTime = Long.MAX_VALUE; + firstValue = null; } @Override public void aggregate() { long time = timeSelector.getLong(); - if (time < lastTime) { - lastTime = time; + if (time < firstTime) { + firstTime = time; Object value = valueSelector.getObject(); if (value instanceof String) { - lastValue = (String) valueSelector.getObject(); + firstValue = (String) value; - if (lastValue.length() > maxStringBytes) { - lastValue = lastValue.substring(0, maxStringBytes); + if (firstValue.length() > maxStringBytes) { + firstValue = firstValue.substring(0, maxStringBytes); } } else if (value instanceof SerializablePair) { - lastValue = ((SerializablePair) value).rhs; + firstValue = ((SerializablePair) value).rhs; } } } @@ -71,7 +71,7 @@ public void aggregate() @Override public Object get() { - return new SerializablePair<>(lastTime, lastValue); + return new SerializablePair<>(firstTime, firstValue); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 44ede0781a2c..62053f36ec03 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -21,8 +21,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.AggregateCombiner; @@ -30,19 +30,17 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; -import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import io.druid.segment.BaseObjectColumnValueSelector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; +@JsonTypeName("stringFirst") public class StringFirstAggregatorFactory extends AggregatorFactory { public static final Integer MAX_SIZE_STRING = 1024; @@ -50,14 +48,9 @@ public class StringFirstAggregatorFactory extends AggregatorFactory ? 1 : 0; - public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( - ((SerializablePair) o1).lhs, - ((SerializablePair) o2).lhs - ); - - private final String fieldName; - private final String name; - private final Integer maxStringBytes; + public final String fieldName; + public final String name; + public final Integer maxStringBytes; @JsonCreator public StringFirstAggregatorFactory( @@ -114,58 +107,7 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringFirstAggregatorFactory(name, name, maxStringBytes) - { - @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) - { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); - return new StringFirstAggregator(null, null, maxStringBytes) - { - @Override - public void aggregate() - { - SerializablePair pair = (SerializablePair) selector.getObject(); - if (pair.lhs < lastTime) { - lastTime = pair.lhs; - lastValue = pair.rhs; - } - } - }; - } - - @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) - { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); - return new StringFirstBufferAggregator(null, null, maxStringBytes) - { - @Override - public void aggregate(ByteBuffer buf, int position) - { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - - SerializablePair pair = (SerializablePair) selector.getObject(); - long lastTime = mutationBuffer.getLong(position); - if (pair.lhs < lastTime) { - mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); - - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - } - }; - } - }; + return new StringFirstFoldingAggregatorFactory(name, name, maxStringBytes); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java new file mode 100644 index 000000000000..6b4308b62a3d --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -0,0 +1,96 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query.aggregation.first; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseObjectColumnValueSelector; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +@JsonTypeName("stringFirstFold") +public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFactory +{ + public StringFirstFoldingAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("maxStringBytes") Integer maxStringBytes + ) + { + super(name, fieldName, maxStringBytes); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringFirstAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate() + { + SerializablePair pair = (SerializablePair) selector.getObject(); + if (pair.lhs < firstTime) { + firstTime = pair.lhs; + firstValue = pair.rhs; + } + } + }; + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringFirstBufferAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + SerializablePair pair = (SerializablePair) selector.getObject(); + long lastTime = mutationBuffer.getLong(position); + if (pair.lhs < lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index 61bc387333f4..323c8093bf4f 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -57,7 +57,7 @@ public void aggregate() Object value = valueSelector.getObject(); if (value instanceof String) { - lastValue = (String) valueSelector.getObject(); + lastValue = (String) value; if (lastValue.length() > maxStringBytes) { lastValue = lastValue.substring(0, maxStringBytes); diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index e1ac198aa22e..eb77db0a6c04 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; import io.druid.collections.SerializablePair; @@ -31,19 +32,17 @@ import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; -import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import io.druid.segment.BaseObjectColumnValueSelector; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; +@JsonTypeName("stringLast") public class StringLastAggregatorFactory extends AggregatorFactory { public static final Integer MAX_SIZE_STRING = 1024; @@ -56,9 +55,9 @@ public class StringLastAggregatorFactory extends AggregatorFactory ((SerializablePair) o2).lhs ); - private final String fieldName; - private final String name; - private final Integer maxStringBytes; + public final String fieldName; + public final String name; + public final Integer maxStringBytes; @JsonCreator public StringLastAggregatorFactory( @@ -115,58 +114,7 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringLastAggregatorFactory(name, name, maxStringBytes) - { - @Override - public Aggregator factorize(ColumnSelectorFactory metricFactory) - { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); - return new StringLastAggregator(null, null, maxStringBytes) - { - @Override - public void aggregate() - { - SerializablePair pair = (SerializablePair) selector.getObject(); - if (pair.lhs >= lastTime) { - lastTime = pair.lhs; - lastValue = pair.rhs; - } - } - }; - } - - @Override - public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) - { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); - return new StringLastBufferAggregator(null, null, maxStringBytes) - { - @Override - public void aggregate(ByteBuffer buf, int position) - { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - - SerializablePair pair = (SerializablePair) selector.getObject(); - long lastTime = mutationBuffer.getLong(position); - if (pair.lhs >= lastTime) { - mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); - - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); - } - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - } - }; - } - }; + return new StringLastFoldingAggregatorFactory(name, name, maxStringBytes); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java new file mode 100644 index 000000000000..e80b5e988098 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -0,0 +1,95 @@ +/* + * 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.last; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import io.druid.collections.SerializablePair; +import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.BaseObjectColumnValueSelector; +import io.druid.segment.ColumnSelectorFactory; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +@JsonTypeName("stringLastFold") +public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFactory +{ + public StringLastFoldingAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") final String fieldName, + @JsonProperty("maxStringBytes") Integer maxStringBytes + ) + { + super(name, fieldName, maxStringBytes); + } + + @Override + public Aggregator factorize(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringLastAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate() + { + SerializablePair pair = (SerializablePair) selector.getObject(); + if (pair.lhs >= lastTime) { + lastTime = pair.lhs; + lastValue = pair.rhs; + } + } + }; + } + + @Override + public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) + { + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + return new StringLastBufferAggregator(null, null, maxStringBytes) + { + @Override + public void aggregate(ByteBuffer buf, int position) + { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + SerializablePair pair = (SerializablePair) selector.getObject(); + long lastTime = mutationBuffer.getLong(position); + if (pair.lhs >= lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } +} From 1c3bd6a5a46db30c3a117cbe94ed51bb20a07a65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 1 Jun 2018 09:15:32 +0200 Subject: [PATCH 07/45] Add getMaxStringBytes() method to support JSON serialization --- .../aggregation/first/StringFirstAggregatorFactory.java | 7 ++++++- .../aggregation/last/StringLastAggregatorFactory.java | 6 ++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 62053f36ec03..17218b5ddb09 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -47,7 +47,6 @@ public class StringFirstAggregatorFactory extends AggregatorFactory public static final Comparator VALUE_COMPARATOR = (o1, o2) -> ((SerializablePair) o1).rhs.equals(((SerializablePair) o2).rhs) ? 1 : 0; - public final String fieldName; public final String name; public final Integer maxStringBytes; @@ -142,6 +141,12 @@ public String getFieldName() return fieldName; } + @JsonProperty + public Integer getMaxStringBytes() + { + return maxStringBytes; + } + @Override public List requiredFields() { diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index eb77db0a6c04..5bb65b8a8a5f 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -149,6 +149,12 @@ public String getFieldName() return fieldName; } + @JsonProperty + public Integer getMaxStringBytes() + { + return maxStringBytes; + } + @Override public List requiredFields() { From c75c88f0993bc4bf4a4e4670face0e9b7e429ce2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Wed, 6 Jun 2018 12:33:54 +0200 Subject: [PATCH 08/45] Fix null pointer exception at segment creation phase when the string value is null --- .../aggregation/SerializablePairSerde.java | 49 ++++++++++++++----- .../first/StringFirstBufferAggregator.java | 16 ++++-- .../last/StringLastBufferAggregator.java | 16 ++++-- 3 files changed, 60 insertions(+), 21 deletions(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java index 05af9150b6b3..c3acc9a111d9 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java @@ -90,10 +90,20 @@ public int compare(SerializablePair o1, SerializablePair o2) comparation = -1; } - if (comparation == 0 && o1.rhs.equals(o2.rhs)) { - comparation = 0; - } else if (comparation == 0) { - comparation = -1; + if (comparation == 0) { + if (o1.rhs != null && o2.rhs != null) { + if (o1.rhs.equals(o2.rhs)) { + comparation = 0; + } else { + comparation = -1; + } + } else if (o1.rhs != null) { + comparation = 1; + } else if (o2.rhs != null) { + comparation = -1; + } else { + comparation = 0; + } } return comparation; @@ -113,22 +123,35 @@ public SerializablePair fromByteBuffer(ByteBuffer buffer, int numB Long lhs = readOnlyBuffer.getLong(); Integer stringSize = readOnlyBuffer.getInt(); - byte[] stringBytes = new byte[stringSize]; - readOnlyBuffer.get(stringBytes, 0, stringSize); + String lastString = null; + if (stringSize > 0) { + byte[] stringBytes = new byte[stringSize]; + readOnlyBuffer.get(stringBytes, 0, stringSize); + lastString = new String(stringBytes, StandardCharsets.UTF_8); + } - return new SerializablePair<>(lhs, new String(stringBytes, StandardCharsets.UTF_8)); + return new SerializablePair<>(lhs, lastString); } @Override public byte[] toBytes(SerializablePair val) { String rhsString = (String) val.rhs; - - ByteBuffer bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsString.length()); - bbuf.putLong((Long) val.lhs); - bbuf.putInt(Long.BYTES, rhsString.length()); - bbuf.position(Long.BYTES + Integer.BYTES); - bbuf.put(rhsString.getBytes(StandardCharsets.UTF_8)); + ByteBuffer bbuf; + + + if (rhsString != null) { + byte[] rhsBytes = rhsString.getBytes(StandardCharsets.UTF_8); + bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsBytes.length); + bbuf.putLong((Long) val.lhs); + bbuf.putInt(Long.BYTES, rhsBytes.length); + bbuf.position(Long.BYTES + Integer.BYTES); + bbuf.put(rhsBytes); + } else { + bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES); + bbuf.putLong((Long) val.lhs); + bbuf.putInt(Long.BYTES, 0); + } return bbuf.array(); } diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index b78da5547ccd..173b0cc9c3bc 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -86,10 +86,18 @@ public Object get(ByteBuffer buf, int position) Long timeValue = mutationBuffer.getLong(position); Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); - byte[] valueBytes = new byte[stringSizeBytes]; - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.get(valueBytes, 0, stringSizeBytes); - return new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + SerializablePair serializablePair; + + if (stringSizeBytes > 0) { + byte[] valueBytes = new byte[stringSizeBytes]; + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.get(valueBytes, 0, stringSizeBytes); + serializablePair = new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + } else { + serializablePair = new SerializablePair<>(timeValue, null); + } + + return serializablePair; } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 181ad0e18cd2..9f622f93b454 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -87,10 +87,18 @@ public Object get(ByteBuffer buf, int position) Long timeValue = mutationBuffer.getLong(position); Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); - byte[] valueBytes = new byte[stringSizeBytes]; - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.get(valueBytes, 0, stringSizeBytes); - return new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + SerializablePair serializablePair; + + if (stringSizeBytes > 0) { + byte[] valueBytes = new byte[stringSizeBytes]; + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.get(valueBytes, 0, stringSizeBytes); + serializablePair = new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + } else { + serializablePair = new SerializablePair<>(timeValue, null); + } + + return serializablePair; } @Override From d6719227e44f0ae2907fe6725cb6be92fd2ebe87 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Wed, 6 Jun 2018 19:06:27 +0200 Subject: [PATCH 09/45] Control the valueSelector object class on BufferAggregators --- .../first/StringFirstBufferAggregator.java | 17 +++++++++++++++-- .../last/StringLastBufferAggregator.java | 18 +++++++++++++++--- 2 files changed, 30 insertions(+), 5 deletions(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index 173b0cc9c3bc..42912808642b 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -64,10 +64,23 @@ public void aggregate(ByteBuffer buf, int position) ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); - long time = timeSelector.getLong(); + Object value = valueSelector.getObject(); + + long time; + String lastString; + + if (value instanceof SerializablePair) { + SerializablePair serializablePair = (SerializablePair) value; + time = serializablePair.lhs; + lastString = serializablePair.rhs; + } else { + time = timeSelector.getLong(); + lastString = (String) value; + } + long lastTime = mutationBuffer.getLong(position); if (time < lastTime) { - byte[] valueBytes = ((String) valueSelector.getObject()).getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = lastString.getBytes(StandardCharsets.UTF_8); mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 9f622f93b454..34b5637edab7 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -64,17 +64,29 @@ public void aggregate(ByteBuffer buf, int position) ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); - long time = timeSelector.getLong(); + Object value = valueSelector.getObject(); + + long time; + String lastString; + + if (value instanceof SerializablePair) { + SerializablePair serializablePair = (SerializablePair) value; + time = serializablePair.lhs; + lastString = serializablePair.rhs; + } else { + time = timeSelector.getLong(); + lastString = (String) value; + } + long lastTime = mutationBuffer.getLong(position); if (time >= lastTime) { - byte[] valueBytes = ((String) valueSelector.getObject()).getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = lastString.getBytes(StandardCharsets.UTF_8); mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.put(valueBytes); - } } From 6945bf976e2ffd925832702b92c0be7cdf2bc60d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:10:21 +0200 Subject: [PATCH 10/45] Perform all improvements --- .../FirstLastStringDruidModule.java | 2 +- .../SerializablePairLongString.java | 26 ++++++++ ...a => SerializablePairLongStringSerde.java} | 48 ++++++++------- .../first/StringFirstAggregateCombiner.java | 8 +-- .../first/StringFirstAggregator.java | 16 ++--- .../first/StringFirstAggregatorFactory.java | 59 ++++++++++++------- .../first/StringFirstBufferAggregator.java | 52 ++++++++-------- .../StringFirstFoldingAggregatorFactory.java | 18 +++--- .../last/StringLastAggregator.java | 16 ++--- .../last/StringLastAggregatorFactory.java | 49 ++++++--------- .../last/StringLastBufferAggregator.java | 47 ++++++++------- .../StringLastFoldingAggregatorFactory.java | 18 +++--- .../first/StringFirstAggregationTest.java | 16 ++--- .../first/StringFirstAggregatorTest.java | 4 +- .../first/StringFirstTimeseriesQueryTest.java | 4 +- .../last/StringLastAggregationTest.java | 16 ++--- .../last/StringLastAggregatorTest.java | 4 +- .../last/StringLastTimeseriesQueryTest.java | 4 +- 18 files changed, 221 insertions(+), 186 deletions(-) create mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java rename extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/{SerializablePairSerde.java => SerializablePairLongStringSerde.java} (76%) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java index ee84133f6eba..3587c5a7aa36 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java @@ -54,7 +54,7 @@ public List getJacksonModules() public void configure(Binder binder) { if (ComplexMetrics.getSerdeForType("serializablePairLongString") == null) { - ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairSerde()); + ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairLongStringSerde()); } } } diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java new file mode 100644 index 000000000000..2d45c603ade9 --- /dev/null +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java @@ -0,0 +1,26 @@ +package io.druid.query.aggregation; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.java.util.common.Pair; + +public class SerializablePairLongString extends Pair +{ + @JsonCreator + public SerializablePairLongString(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") String rhs) + { + super(lhs, rhs); + } + + @JsonProperty + public Long getLhs() + { + return lhs; + } + + @JsonProperty + public String getRhs() + { + return rhs; + } +} diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java similarity index 76% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java rename to extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index c3acc9a111d9..3bd9ae885d46 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -19,8 +19,8 @@ package io.druid.query.aggregation; -import io.druid.collections.SerializablePair; import io.druid.data.input.InputRow; +import io.druid.java.util.common.StringUtils; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -32,20 +32,17 @@ import io.druid.segment.writeout.SegmentWriteOutMedium; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -public class SerializablePairSerde extends ComplexMetricSerde +//TODO: JAVA DOCS +public class SerializablePairLongStringSerde extends ComplexMetricSerde { - public SerializablePairSerde() - { - - } + public static final String TYPE_NAME = "serializablePairLongString"; @Override public String getTypeName() { - return "serializablePairLongString"; + return TYPE_NAME; } @Override @@ -54,9 +51,9 @@ public ComplexMetricExtractor getExtractor() return new ComplexMetricExtractor() { @Override - public Class extractedClass() + public Class extractedClass() { - return SerializablePair.class; + return SerializablePairLongString.class; } @Override @@ -77,16 +74,17 @@ public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder) @Override public ObjectStrategy getObjectStrategy() { - return new ObjectStrategy() + return new ObjectStrategy() { @Override - public int compare(SerializablePair o1, SerializablePair o2) + public int compare(SerializablePairLongString o1, SerializablePairLongString o2) { - Integer comparation = 0; + //TODO: DOCS + int comparation = 0; - if ((Long) o1.lhs > (Long) o2.lhs) { + if (o1.lhs > o2.lhs) { comparation = 1; - } else if ((Long) o1.lhs < (Long) o2.lhs) { + } else if (o1.lhs < o2.lhs) { comparation = -1; } @@ -110,13 +108,13 @@ public int compare(SerializablePair o1, SerializablePair o2) } @Override - public Class getClazz() + public Class getClazz() { - return SerializablePair.class; + return SerializablePairLongString.class; } @Override - public SerializablePair fromByteBuffer(ByteBuffer buffer, int numBytes) + public SerializablePairLongString fromByteBuffer(ByteBuffer buffer, int numBytes) { final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); @@ -127,29 +125,29 @@ public SerializablePair fromByteBuffer(ByteBuffer buffer, int numB if (stringSize > 0) { byte[] stringBytes = new byte[stringSize]; readOnlyBuffer.get(stringBytes, 0, stringSize); - lastString = new String(stringBytes, StandardCharsets.UTF_8); + lastString = StringUtils.fromUtf8(stringBytes); } - return new SerializablePair<>(lhs, lastString); + return new SerializablePairLongString(lhs, lastString); } @Override - public byte[] toBytes(SerializablePair val) + public byte[] toBytes(SerializablePairLongString val) { - String rhsString = (String) val.rhs; + String rhsString = val.rhs; ByteBuffer bbuf; if (rhsString != null) { - byte[] rhsBytes = rhsString.getBytes(StandardCharsets.UTF_8); + byte[] rhsBytes = StringUtils.toUtf8(rhsString); bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsBytes.length); - bbuf.putLong((Long) val.lhs); + bbuf.putLong(val.lhs); bbuf.putInt(Long.BYTES, rhsBytes.length); bbuf.position(Long.BYTES + Integer.BYTES); bbuf.put(rhsBytes); } else { bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES); - bbuf.putLong((Long) val.lhs); + bbuf.putLong(val.lhs); bbuf.putInt(Long.BYTES, 0); } diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java index 2811847f1bab..b694dd0dd9ab 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -26,25 +26,25 @@ public class StringFirstAggregateCombiner extends ObjectAggregateCombiner { - String lastString; + private String firstString; @Override public void reset(ColumnValueSelector selector) { - lastString = (String) selector.getObject(); + firstString = (String) selector.getObject(); } @Override public void fold(ColumnValueSelector selector) { - lastString = (String) selector.getObject(); + // Nothing to do. It needs to keep the first string value. } @Nullable @Override public String getObject() { - return lastString; + return firstString; } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index 6a4fd40931f0..5d2cf014d1e6 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -19,8 +19,8 @@ package io.druid.query.aggregation.first; -import io.druid.collections.SerializablePair; import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.segment.BaseLongColumnValueSelector; import io.druid.segment.BaseObjectColumnValueSelector; @@ -58,12 +58,14 @@ public void aggregate() if (value instanceof String) { firstValue = (String) value; + } else if (value instanceof SerializablePairLongString) { + firstValue = ((SerializablePairLongString) value).rhs; + } else if (value != null) { + firstValue = value.toString(); + } - if (firstValue.length() > maxStringBytes) { - firstValue = firstValue.substring(0, maxStringBytes); - } - } else if (value instanceof SerializablePair) { - firstValue = ((SerializablePair) value).rhs; + if (firstValue != null && firstValue.length() > maxStringBytes) { + firstValue = firstValue.substring(0, maxStringBytes); } } } @@ -71,7 +73,7 @@ public void aggregate() @Override public Object get() { - return new SerializablePair<>(firstTime, firstValue); + return new SerializablePairLongString(firstTime, firstValue); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 17218b5ddb09..7d7d77307483 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -23,17 +23,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import io.druid.collections.SerializablePair; -import io.druid.java.util.common.StringUtils; +import com.google.common.primitives.Longs; import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -43,13 +43,30 @@ @JsonTypeName("stringFirst") public class StringFirstAggregatorFactory extends AggregatorFactory { - public static final Integer MAX_SIZE_STRING = 1024; - public static final Comparator VALUE_COMPARATOR = (o1, o2) -> ((SerializablePair) o1).rhs.equals(((SerializablePair) o2).rhs) - ? 1 - : 0; - public final String fieldName; - public final String name; - public final Integer maxStringBytes; + public static final int DEFAULT_MAX_STRING_SIZE = 1024; + + public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( + ((SerializablePairLongString) o1).lhs, + ((SerializablePairLongString) o2).lhs + ); + + public static final Comparator VALUE_COMPARATOR = (o1, o2) -> { + String s1 = null; + String s2 = null; + + if (o1 != null) { + s1 = ((SerializablePairLongString) o1).rhs; + } + if (o2 != null) { + s2 = ((SerializablePairLongString) o2).rhs; + } + + return Objects.equals(s1, s2) ? 1 : 0; + }; + + private final String fieldName; + private final String name; + protected final int maxStringBytes; @JsonCreator public StringFirstAggregatorFactory( @@ -62,7 +79,7 @@ public StringFirstAggregatorFactory( Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); this.name = name; this.fieldName = fieldName; - this.maxStringBytes = maxStringBytes == null ? MAX_SIZE_STRING : maxStringBytes; + this.maxStringBytes = maxStringBytes == null ? DEFAULT_MAX_STRING_SIZE : maxStringBytes; } @Override @@ -94,7 +111,7 @@ public Comparator getComparator() @Override public Object combine(Object lhs, Object rhs) { - return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; + return TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; } @Override @@ -119,13 +136,13 @@ public List getRequiredColumns() public Object deserialize(Object object) { Map map = (Map) object; - return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); + return new SerializablePairLongString(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); } @Override public Object finalizeComputation(Object object) { - return ((SerializablePair) object).rhs; + return ((SerializablePairLongString) object).rhs; } @Override @@ -156,12 +173,10 @@ public List requiredFields() @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - - return ByteBuffer.allocate(1 + fieldNameBytes.length) - .put(AggregatorUtil.STRING_FIRST_CACHE_TYPE_ID) - .put(fieldNameBytes) - .array(); + return new CacheKeyBuilder(AggregatorUtil.STRING_FIRST_CACHE_TYPE_ID) + .appendString(fieldName) + .appendInt(maxStringBytes) + .build(); } @Override @@ -188,13 +203,13 @@ public boolean equals(Object o) StringFirstAggregatorFactory that = (StringFirstAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name); + return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; } @Override public int hashCode() { - return Objects.hash(name, fieldName); + return Objects.hash(name, fieldName, maxStringBytes); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index 42912808642b..2802b377b6c3 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -19,25 +19,26 @@ package io.druid.query.aggregation.first; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.BaseLongColumnValueSelector; import io.druid.segment.BaseObjectColumnValueSelector; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; +// TODO: Unit Test public class StringFirstBufferAggregator implements BufferAggregator { private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; - private final Integer maxStringBytes; + private final int maxStringBytes; public StringFirstBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - Integer maxStringBytes + int maxStringBytes ) { this.timeSelector = timeSelector; @@ -48,14 +49,8 @@ public StringFirstBufferAggregator( @Override public void init(ByteBuffer buf, int position) { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - - mutationBuffer.putLong(position, Long.MAX_VALUE); - mutationBuffer.putInt(position + Long.BYTES, 0); - for (int i = 0; i < maxStringBytes - 1; i++) { - mutationBuffer.putChar(position + Long.BYTES + Integer.BYTES + i, '\0'); - } + buf.putLong(position, Long.MAX_VALUE); + buf.putInt(position + Long.BYTES, 0); } @Override @@ -66,21 +61,28 @@ public void aggregate(ByteBuffer buf, int position) Object value = valueSelector.getObject(); - long time; - String lastString; + long time = Long.MAX_VALUE; + String firstString = null; - if (value instanceof SerializablePair) { - SerializablePair serializablePair = (SerializablePair) value; + if (value instanceof SerializablePairLongString) { + SerializablePairLongString serializablePair = (SerializablePairLongString) value; time = serializablePair.lhs; - lastString = serializablePair.rhs; - } else { + firstString = serializablePair.rhs; + } else if (value instanceof String) { + time = timeSelector.getLong(); + firstString = (String) value; + } else if (value != null) { time = timeSelector.getLong(); - lastString = (String) value; + firstString = value.toString(); } long lastTime = mutationBuffer.getLong(position); - if (time < lastTime) { - byte[] valueBytes = lastString.getBytes(StandardCharsets.UTF_8); + if (firstString != null && time < lastTime) { + if (firstString.length() > maxStringBytes) { + firstString = firstString.substring(0, maxStringBytes); + } + + byte[] valueBytes = StringUtils.toUtf8(firstString); mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); @@ -97,17 +99,17 @@ public Object get(ByteBuffer buf, int position) mutationBuffer.position(position); Long timeValue = mutationBuffer.getLong(position); - Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); + int stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); - SerializablePair serializablePair; + SerializablePairLongString serializablePair; if (stringSizeBytes > 0) { byte[] valueBytes = new byte[stringSizeBytes]; mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.get(valueBytes, 0, stringSizeBytes); - serializablePair = new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + serializablePair = new SerializablePairLongString(timeValue, StringUtils.fromUtf8(valueBytes)); } else { - serializablePair = new SerializablePair<>(timeValue, null); + serializablePair = new SerializablePairLongString(timeValue, null); } return serializablePair; diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index 6b4308b62a3d..c1079752469a 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -21,15 +21,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.BaseObjectColumnValueSelector; import io.druid.segment.ColumnSelectorFactory; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; @JsonTypeName("stringFirstFold") public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFactory @@ -46,14 +46,14 @@ public StringFirstFoldingAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); return new StringFirstAggregator(null, null, maxStringBytes) { @Override public void aggregate() { - SerializablePair pair = (SerializablePair) selector.getObject(); - if (pair.lhs < firstTime) { + SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); + if (pair != null && pair.lhs < firstTime) { firstTime = pair.lhs; firstValue = pair.rhs; } @@ -64,7 +64,7 @@ public void aggregate() @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); return new StringFirstBufferAggregator(null, null, maxStringBytes) { @Override @@ -73,11 +73,11 @@ public void aggregate(ByteBuffer buf, int position) ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); - SerializablePair pair = (SerializablePair) selector.getObject(); + SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); long lastTime = mutationBuffer.getLong(position); - if (pair.lhs < lastTime) { + if (pair != null && pair.lhs < lastTime) { mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); mutationBuffer.position(position + Long.BYTES + Integer.BYTES); diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index 323c8093bf4f..c93316a9c00c 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -19,8 +19,8 @@ package io.druid.query.aggregation.last; -import io.druid.collections.SerializablePair; import io.druid.query.aggregation.Aggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.segment.BaseLongColumnValueSelector; import io.druid.segment.BaseObjectColumnValueSelector; @@ -58,12 +58,14 @@ public void aggregate() if (value instanceof String) { lastValue = (String) value; + } else if (value instanceof SerializablePairLongString) { + lastValue = ((SerializablePairLongString) value).rhs; + } else if (value != null) { + lastValue = value.toString(); + } - if (lastValue.length() > maxStringBytes) { - lastValue = lastValue.substring(0, maxStringBytes); - } - } else if (value instanceof SerializablePair) { - lastValue = ((SerializablePair) value).rhs; + if (lastValue != null && lastValue.length() > maxStringBytes) { + lastValue = lastValue.substring(0, maxStringBytes); } } } @@ -71,7 +73,7 @@ public void aggregate() @Override public Object get() { - return new SerializablePair<>(lastTime, lastValue); + return new SerializablePairLongString(lastTime, lastValue); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index 5bb65b8a8a5f..853037ab7cc1 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -23,41 +23,30 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; -import com.google.common.primitives.Longs; -import io.druid.collections.SerializablePair; -import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorUtil; import io.druid.query.aggregation.BufferAggregator; -import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; +import io.druid.query.aggregation.SerializablePairLongString; +import io.druid.query.aggregation.first.StringFirstAggregatorFactory; +import io.druid.query.cache.CacheKeyBuilder; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; + @JsonTypeName("stringLast") public class StringLastAggregatorFactory extends AggregatorFactory { - public static final Integer MAX_SIZE_STRING = 1024; - public static final Comparator VALUE_COMPARATOR = (o1, o2) -> ((SerializablePair) o1).rhs.equals(((SerializablePair) o2).rhs) - ? 1 - : 0; - - public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( - ((SerializablePair) o1).lhs, - ((SerializablePair) o2).lhs - ); - - public final String fieldName; - public final String name; - public final Integer maxStringBytes; + private final String fieldName; + private final String name; + protected final int maxStringBytes; @JsonCreator public StringLastAggregatorFactory( @@ -70,7 +59,7 @@ public StringLastAggregatorFactory( Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName"); this.name = name; this.fieldName = fieldName; - this.maxStringBytes = maxStringBytes == null ? MAX_SIZE_STRING : maxStringBytes; + this.maxStringBytes = maxStringBytes == null ? StringFirstAggregatorFactory.DEFAULT_MAX_STRING_SIZE : maxStringBytes; } @Override @@ -96,13 +85,13 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) @Override public Comparator getComparator() { - return VALUE_COMPARATOR; + return StringFirstAggregatorFactory.VALUE_COMPARATOR; } @Override public Object combine(Object lhs, Object rhs) { - return DoubleFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; + return StringFirstAggregatorFactory.TIME_COMPARATOR.compare(lhs, rhs) > 0 ? lhs : rhs; } @Override @@ -127,13 +116,13 @@ public List getRequiredColumns() public Object deserialize(Object object) { Map map = (Map) object; - return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); + return new SerializablePairLongString(((Number) map.get("lhs")).longValue(), ((String) map.get("rhs"))); } @Override public Object finalizeComputation(Object object) { - return ((SerializablePair) object).rhs; + return ((SerializablePairLongString) object).rhs; } @Override @@ -164,12 +153,10 @@ public List requiredFields() @Override public byte[] getCacheKey() { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - - return ByteBuffer.allocate(1 + fieldNameBytes.length) - .put(AggregatorUtil.STRING_LAST_CACHE_TYPE_ID) - .put(fieldNameBytes) - .array(); + return new CacheKeyBuilder(AggregatorUtil.STRING_LAST_CACHE_TYPE_ID) + .appendString(fieldName) + .appendInt(maxStringBytes) + .build(); } @Override @@ -196,13 +183,13 @@ public boolean equals(Object o) StringLastAggregatorFactory that = (StringLastAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name); + return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; } @Override public int hashCode() { - return Objects.hash(name, fieldName); + return Objects.hash(name, fieldName, maxStringBytes); } @Override diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 34b5637edab7..89ecebb7d944 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -19,25 +19,26 @@ package io.druid.query.aggregation.last; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.BaseLongColumnValueSelector; import io.druid.segment.BaseObjectColumnValueSelector; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; +//TODO: Unit Test public class StringLastBufferAggregator implements BufferAggregator { private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; - private final Integer maxStringBytes; + private final int maxStringBytes; public StringLastBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - Integer maxStringBytes + int maxStringBytes ) { this.timeSelector = timeSelector; @@ -48,14 +49,8 @@ public StringLastBufferAggregator( @Override public void init(ByteBuffer buf, int position) { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - - mutationBuffer.putLong(position, Long.MIN_VALUE); - mutationBuffer.putInt(position + Long.BYTES, 0); - for (int i = 0; i < maxStringBytes - 1; i++) { - mutationBuffer.putChar(position + Long.BYTES + Integer.BYTES + i, '\0'); - } + buf.putLong(position, Long.MIN_VALUE); + buf.putInt(position + Long.BYTES, 0); } @Override @@ -66,21 +61,29 @@ public void aggregate(ByteBuffer buf, int position) Object value = valueSelector.getObject(); - long time; - String lastString; + long time = Long.MIN_VALUE; + ; + String lastString = null; - if (value instanceof SerializablePair) { - SerializablePair serializablePair = (SerializablePair) value; + if (value instanceof SerializablePairLongString) { + SerializablePairLongString serializablePair = (SerializablePairLongString) value; time = serializablePair.lhs; lastString = serializablePair.rhs; - } else { + } else if (value instanceof String) { time = timeSelector.getLong(); lastString = (String) value; + } else if (value != null) { + time = timeSelector.getLong(); + lastString = value.toString(); } long lastTime = mutationBuffer.getLong(position); - if (time >= lastTime) { - byte[] valueBytes = lastString.getBytes(StandardCharsets.UTF_8); + if (lastString != null && time >= lastTime) { + if (lastString.length() > maxStringBytes) { + lastString = lastString.substring(0, maxStringBytes); + } + + byte[] valueBytes = StringUtils.toUtf8(lastString); mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); @@ -99,15 +102,15 @@ public Object get(ByteBuffer buf, int position) Long timeValue = mutationBuffer.getLong(position); Integer stringSizeBytes = mutationBuffer.getInt(position + Long.BYTES); - SerializablePair serializablePair; + SerializablePairLongString serializablePair; if (stringSizeBytes > 0) { byte[] valueBytes = new byte[stringSizeBytes]; mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.get(valueBytes, 0, stringSizeBytes); - serializablePair = new SerializablePair<>(timeValue, new String(valueBytes, StandardCharsets.UTF_8)); + serializablePair = new SerializablePairLongString(timeValue, StringUtils.fromUtf8(valueBytes)); } else { - serializablePair = new SerializablePair<>(timeValue, null); + serializablePair = new SerializablePairLongString(timeValue, null); } return serializablePair; diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index e80b5e988098..c4b346e46516 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -21,15 +21,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.BaseObjectColumnValueSelector; import io.druid.segment.ColumnSelectorFactory; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; @JsonTypeName("stringLastFold") public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFactory @@ -46,14 +46,14 @@ public StringLastFoldingAggregatorFactory( @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); return new StringLastAggregator(null, null, maxStringBytes) { @Override public void aggregate() { - SerializablePair pair = (SerializablePair) selector.getObject(); - if (pair.lhs >= lastTime) { + SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); + if (pair != null && pair.lhs >= lastTime) { lastTime = pair.lhs; lastValue = pair.rhs; } @@ -64,7 +64,7 @@ public void aggregate() @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { - final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(name); + final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); return new StringLastBufferAggregator(null, null, maxStringBytes) { @Override @@ -73,11 +73,11 @@ public void aggregate(ByteBuffer buf, int position) ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); - SerializablePair pair = (SerializablePair) selector.getObject(); + SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); long lastTime = mutationBuffer.getLong(position); - if (pair.lhs >= lastTime) { + if (pair != null && pair.lhs >= lastTime) { mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = pair.rhs.getBytes(StandardCharsets.UTF_8); + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); mutationBuffer.position(position + Long.BYTES + Integer.BYTES); diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 857aa0511005..839b647539d8 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -19,9 +19,9 @@ package io.druid.query.aggregation.first; -import io.druid.collections.SerializablePair; import io.druid.java.util.common.Pair; import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; import io.druid.segment.ColumnSelectorFactory; @@ -45,11 +45,11 @@ public class StringFirstAggregationTest private String[] strings = {"1111", "2222", "3333", "4444"}; private long[] times = {8224, 6879, 2436, 7888}; - private SerializablePair[] pairs = { - new SerializablePair<>(52782L, "AAAA"), - new SerializablePair<>(65492L, "BBBB"), - new SerializablePair<>(69134L, "CCCC"), - new SerializablePair<>(11111L, "DDDD") + private SerializablePairLongString[] pairs = { + new SerializablePairLongString(52782L, "AAAA"), + new SerializablePairLongString(65492L, "BBBB"), + new SerializablePairLongString(69134L, "CCCC"), + new SerializablePairLongString(11111L, "DDDD") }; @Before @@ -107,8 +107,8 @@ public void testStringLastBufferAggregator() @Test public void testCombine() { - SerializablePair pair1 = new SerializablePair<>(1467225000L, "AAAA"); - SerializablePair pair2 = new SerializablePair<>(1467240000L, "BBBB"); + SerializablePairLongString pair1 = new SerializablePairLongString(1467225000L, "AAAA"); + SerializablePairLongString pair2 = new SerializablePairLongString(1467240000L, "BBBB"); Assert.assertEquals(pair2, stringLastAggFactory.combine(pair1, pair2)); } diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java index bc2c81dc3c3f..80d7c3472292 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java @@ -19,8 +19,8 @@ package io.druid.query.aggregation.first; -import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; import io.druid.query.aggregation.last.StringLastAggregatorFactory; @@ -82,7 +82,7 @@ public void testBufferAggregate() throws Exception aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); } - SerializablePair sp = ((SerializablePair) agg.get(buf, position)); + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); Assert.assertEquals("expectec last string value", "DDDD", sp.rhs); diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index 446d0c34c092..d33d6d797867 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.collections.SerializablePair; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.granularity.Granularities; @@ -30,6 +29,7 @@ import io.druid.query.Result; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; import io.druid.query.timeseries.TimeseriesResultValue; @@ -121,7 +121,7 @@ public void testTopNWithDistinctCountAgg() throws Exception new Result<>( time, new TimeseriesResultValue( - ImmutableMap.of("last_client_type", new SerializablePair<>(timestamp, "iphone")) + ImmutableMap.of("last_client_type", new SerializablePairLongString(timestamp, "iphone")) ) ) ); diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index c5bc7c7c53a9..d00a0a60f769 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -19,9 +19,9 @@ package io.druid.query.aggregation.last; -import io.druid.collections.SerializablePair; import io.druid.java.util.common.Pair; import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; import io.druid.segment.ColumnSelectorFactory; @@ -45,11 +45,11 @@ public class StringLastAggregationTest private String[] strings = {"1111", "2222", "3333", "4444"}; private long[] times = {8224, 6879, 2436, 7888}; - private SerializablePair[] pairs = { - new SerializablePair<>(52782L, "AAAA"), - new SerializablePair<>(65492L, "BBBB"), - new SerializablePair<>(69134L, "CCCC"), - new SerializablePair<>(11111L, "DDDD") + private SerializablePairLongString[] pairs = { + new SerializablePairLongString(52782L, "AAAA"), + new SerializablePairLongString(65492L, "BBBB"), + new SerializablePairLongString(69134L, "CCCC"), + new SerializablePairLongString(11111L, "DDDD") }; @Before @@ -107,8 +107,8 @@ public void testStringLastBufferAggregator() @Test public void testCombine() { - SerializablePair pair1 = new SerializablePair<>(1467225000L, "AAAA"); - SerializablePair pair2 = new SerializablePair<>(1467240000L, "BBBB"); + SerializablePairLongString pair1 = new SerializablePairLongString(1467225000L, "AAAA"); + SerializablePairLongString pair2 = new SerializablePairLongString(1467240000L, "BBBB"); Assert.assertEquals(pair2, stringLastAggFactory.combine(pair1, pair2)); } diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java index aea6a53784e5..49d07cc1c0a7 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java @@ -19,8 +19,8 @@ package io.druid.query.aggregation.last; -import io.druid.collections.SerializablePair; import io.druid.query.aggregation.BufferAggregator; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; import org.junit.Assert; @@ -80,7 +80,7 @@ public void testBufferAggregate() throws Exception aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); } - SerializablePair sp = ((SerializablePair) agg.get(buf, position)); + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); Assert.assertEquals("expectec last string value", "DDDD", sp.rhs); diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 11e88256fe41..339d2ab4e84c 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.collections.SerializablePair; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.granularity.Granularities; @@ -30,6 +29,7 @@ import io.druid.query.Result; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.FirstLastStringDruidModule; +import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; import io.druid.query.timeseries.TimeseriesResultValue; @@ -121,7 +121,7 @@ public void testTopNWithDistinctCountAgg() throws Exception new Result<>( time, new TimeseriesResultValue( - ImmutableMap.of("last_client_type", new SerializablePair<>(timestamp1, "android")) + ImmutableMap.of("last_client_type", new SerializablePairLongString(timestamp1, "android")) ) ) ); From 7dd12e04107700deed19b8167f0836c31f562040 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:15:50 +0200 Subject: [PATCH 11/45] Add java doc on SerializablePairLongStringSerde --- .../aggregation/SerializablePairLongStringSerde.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 3bd9ae885d46..6c6b87784a3d 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -33,7 +33,13 @@ import java.nio.ByteBuffer; -//TODO: JAVA DOCS +/** + * The SerializablePairLongStringSerde serializes a Long-String pair. + * The serialization structure is: Long:Integer:String + * + * The class is used on first/last String aggregators to store the time and the first/last string. + * Long:Integer:String -> Timestamp:StringSize:StringData + */ public class SerializablePairLongStringSerde extends ComplexMetricSerde { From 217627f6104145a23e8033442cefb6469104b3fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:28:27 +0200 Subject: [PATCH 12/45] Refactor ObjectStraty compare method --- .../SerializablePairLongStringSerde.java | 42 +++++++++++-------- 1 file changed, 25 insertions(+), 17 deletions(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 6c6b87784a3d..de13b36dca49 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -36,7 +36,7 @@ /** * The SerializablePairLongStringSerde serializes a Long-String pair. * The serialization structure is: Long:Integer:String - * + *

* The class is used on first/last String aggregators to store the time and the first/last string. * Long:Integer:String -> Timestamp:StringSize:StringData */ @@ -85,28 +85,36 @@ public ObjectStrategy getObjectStrategy() @Override public int compare(SerializablePairLongString o1, SerializablePairLongString o2) { - //TODO: DOCS - int comparation = 0; + int comparation; - if (o1.lhs > o2.lhs) { - comparation = 1; - } else if (o1.lhs < o2.lhs) { + // First we check if the objects are null + if (o1 == null && o2 == null) { + comparation = 0; + } else if (o1 == null) { comparation = -1; - } + } else if (o2 == null) { + comparation = 1; + } else { - if (comparation == 0) { - if (o1.rhs != null && o2.rhs != null) { - if (o1.rhs.equals(o2.rhs)) { + // If the objects are not null, we will try to compare using timestamp + comparation = o1.lhs.compareTo(o2.lhs); + + // If both timestamp are the same, we try to compare the Strings + if (comparation == 0) { + + // First we check if the strings are null + if (o1.rhs == null && o2.rhs == null) { comparation = 0; - } else { + } else if (o1.rhs == null) { comparation = -1; + } else if (o2.rhs == null) { + comparation = 1; + } else { + + // If the strings are not null, we will compare them + // Note: This comparation maybe doesn't make sense to first/last aggregators + comparation = o1.rhs.compareTo(o2.rhs); } - } else if (o1.rhs != null) { - comparation = 1; - } else if (o2.rhs != null) { - comparation = -1; - } else { - comparation = 0; } } From 9b68a605ce9ff950bd59ba20d4b700ba74f589fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:48:44 +0200 Subject: [PATCH 13/45] Remove unused ; --- .../druid/query/aggregation/last/StringLastBufferAggregator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 89ecebb7d944..0458cfec3fa9 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -62,7 +62,6 @@ public void aggregate(ByteBuffer buf, int position) Object value = valueSelector.getObject(); long time = Long.MIN_VALUE; - ; String lastString = null; if (value instanceof SerializablePairLongString) { From e552f049ad06b43ce438dd0effab4522facc568a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:49:26 +0200 Subject: [PATCH 14/45] Add aggregateCombiner unit tests. Rename BufferAggregators unit tests --- .../first/StringFirstAggregationTest.java | 29 +++++++++++++++++-- ...a => StringFirstBufferAggregatorTest.java} | 12 ++++---- .../last/StringLastAggregationTest.java | 24 +++++++++++++++ ...va => StringLastBufferAggregatorTest.java} | 2 +- 4 files changed, 56 insertions(+), 11 deletions(-) rename extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/{StringFirstAggregatorTest.java => StringFirstBufferAggregatorTest.java} (86%) rename extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/{StringLastAggregatorTest.java => StringLastBufferAggregatorTest.java} (98%) diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 839b647539d8..58a8a4a2720b 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -123,14 +123,14 @@ public void testStringLastCombiningAggregator() aggregate(agg); Pair result = (Pair) agg.get(); - Pair expected = (Pair) pairs[3]; + Pair expected = pairs[3]; Assert.assertEquals(expected.lhs, result.lhs); Assert.assertEquals(expected.rhs, result.rhs); } @Test - public void testStringLastCombiningBufferAggregator() + public void testStringFirstCombiningBufferAggregator() { StringFirstBufferAggregator agg = (StringFirstBufferAggregator) combiningAggFactory.factorizeBuffered( colSelectorFactory); @@ -144,12 +144,35 @@ public void testStringLastCombiningBufferAggregator() aggregate(agg, buffer, 0); Pair result = (Pair) agg.get(buffer, 0); - Pair expected = (Pair) pairs[3]; + Pair expected = pairs[3]; Assert.assertEquals(expected.lhs, result.lhs); Assert.assertEquals(expected.rhs, result.rhs); } + @Test + public void testStringFirstAggregateCombiner() + { + final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; + TestObjectColumnSelector columnSelector = new TestObjectColumnSelector<>(strings); + + StringFirstAggregateCombiner stringFirstAggregateCombiner = + (StringFirstAggregateCombiner) combiningAggFactory.makeAggregateCombiner(); + + stringFirstAggregateCombiner.reset(columnSelector); + + Assert.assertEquals(strings[0], stringFirstAggregateCombiner.getObject()); + + columnSelector.increment(); + stringFirstAggregateCombiner.fold(columnSelector); + + Assert.assertEquals(strings[0], stringFirstAggregateCombiner.getObject()); + + stringFirstAggregateCombiner.reset(columnSelector); + + Assert.assertEquals(strings[1], stringFirstAggregateCombiner.getObject()); + } + private void aggregate( StringFirstAggregator agg ) diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java similarity index 86% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java rename to extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index 80d7c3472292..43a0adb76dd3 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregatorTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -23,15 +23,13 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; -import io.druid.query.aggregation.last.StringLastAggregatorFactory; -import io.druid.query.aggregation.last.StringLastBufferAggregator; import org.junit.Assert; import org.junit.Test; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -public class StringFirstAggregatorTest +public class StringFirstBufferAggregatorTest { private void aggregateBuffer( TestLongColumnSelector timeSelector, @@ -57,11 +55,11 @@ public void testBufferAggregate() throws Exception TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); - StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( "billy", "billy", maxStringBytes ); - StringLastBufferAggregator agg = new StringLastBufferAggregator( + StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, maxStringBytes @@ -85,8 +83,8 @@ public void testBufferAggregate() throws Exception SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - Assert.assertEquals("expectec last string value", "DDDD", sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(1526725900L), new Long(sp.lhs)); + Assert.assertEquals("expectec last string value", strings[0], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[0]), new Long(sp.lhs)); } diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index d00a0a60f769..4ff7c5c41ee4 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -24,6 +24,7 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.query.aggregation.first.StringFirstAggregateCombiner; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; import org.easymock.EasyMock; @@ -150,6 +151,29 @@ public void testStringLastCombiningBufferAggregator() Assert.assertEquals(expected.rhs, result.rhs); } + @Test + public void testStringLastAggregateCombiner() + { + final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; + TestObjectColumnSelector columnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregateCombiner stringFirstAggregateCombiner = + (StringLastAggregateCombiner) combiningAggFactory.makeAggregateCombiner(); + + stringFirstAggregateCombiner.reset(columnSelector); + + Assert.assertEquals(strings[0], stringFirstAggregateCombiner.getObject()); + + columnSelector.increment(); + stringFirstAggregateCombiner.fold(columnSelector); + + Assert.assertEquals(strings[1], stringFirstAggregateCombiner.getObject()); + + stringFirstAggregateCombiner.reset(columnSelector); + + Assert.assertEquals(strings[1], stringFirstAggregateCombiner.getObject()); + } + private void aggregate( StringLastAggregator agg ) diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java similarity index 98% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java rename to extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index 49d07cc1c0a7..af82fb1208d0 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregatorTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -29,7 +29,7 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -public class StringLastAggregatorTest +public class StringLastBufferAggregatorTest { private void aggregateBuffer( TestLongColumnSelector timeSelector, From 48325f399bdbf1b32f9d8b8a72a0398df6e6dbb7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:53:13 +0200 Subject: [PATCH 15/45] Remove unused imports --- .../druid/query/aggregation/last/StringLastAggregationTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index 4ff7c5c41ee4..01d32af17508 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -24,7 +24,6 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; -import io.druid.query.aggregation.first.StringFirstAggregateCombiner; import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.column.Column; import org.easymock.EasyMock; From 657e8a3a54062b4c572bae17e4a18e5f211fd812 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:53:25 +0200 Subject: [PATCH 16/45] Add license header --- .../SerializablePairLongString.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java index 2d45c603ade9..20eb1dad9ac6 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java @@ -1,3 +1,22 @@ +/* + * 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; From e8a2dedbabcb7a7b4dfc7396c043a4c971282937 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sat, 9 Jun 2018 15:53:49 +0200 Subject: [PATCH 17/45] Add class name to java doc class serde --- .../query/aggregation/SerializablePairLongStringSerde.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index de13b36dca49..2aa29fd2c464 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -34,7 +34,7 @@ import java.nio.ByteBuffer; /** - * The SerializablePairLongStringSerde serializes a Long-String pair. + * The SerializablePairLongStringSerde serializes a Long-String pair (SerializablePairLongString). * The serialization structure is: Long:Integer:String *

* The class is used on first/last String aggregators to store the time and the first/last string. From 4d241599bbcc3f9b021815b0bd074ccff7977126 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 00:45:45 +0200 Subject: [PATCH 18/45] Throw exception if value is unsupported class type --- .../query/aggregation/first/StringFirstAggregator.java | 6 +++++- .../aggregation/first/StringFirstBufferAggregator.java | 7 +++++-- .../druid/query/aggregation/last/StringLastAggregator.java | 6 +++++- .../query/aggregation/last/StringLastBufferAggregator.java | 7 +++++-- 4 files changed, 20 insertions(+), 6 deletions(-) diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index 5d2cf014d1e6..e20b976f76a4 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -61,7 +61,11 @@ public void aggregate() } else if (value instanceof SerializablePairLongString) { firstValue = ((SerializablePairLongString) value).rhs; } else if (value != null) { - firstValue = value.toString(); + throw new IllegalStateException( + "Try to aggregate unsuported class type [" + + value.getClass().getName() + + "]. Supported class types: String or SerializablePairLongString" + ); } if (firstValue != null && firstValue.length() > maxStringBytes) { diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index 2802b377b6c3..47bd0d437796 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -72,8 +72,11 @@ public void aggregate(ByteBuffer buf, int position) time = timeSelector.getLong(); firstString = (String) value; } else if (value != null) { - time = timeSelector.getLong(); - firstString = value.toString(); + throw new IllegalStateException( + "Try to aggregate unsuported class type [" + + value.getClass().getName() + + "]. Supported class types: String or SerializablePairLongString" + ); } long lastTime = mutationBuffer.getLong(position); diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index c93316a9c00c..81b90143990d 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -61,7 +61,11 @@ public void aggregate() } else if (value instanceof SerializablePairLongString) { lastValue = ((SerializablePairLongString) value).rhs; } else if (value != null) { - lastValue = value.toString(); + throw new IllegalStateException( + "Try to aggregate unsuported class type [" + + value.getClass().getName() + + "]. Supported class types: String or SerializablePairLongString" + ); } if (lastValue != null && lastValue.length() > maxStringBytes) { diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 0458cfec3fa9..aabe04634295 100644 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -72,8 +72,11 @@ public void aggregate(ByteBuffer buf, int position) time = timeSelector.getLong(); lastString = (String) value; } else if (value != null) { - time = timeSelector.getLong(); - lastString = value.toString(); + throw new IllegalStateException( + "Try to aggregate unsuported class type [" + + value.getClass().getName() + + "]. Supported class types: String or SerializablePairLongString" + ); } long lastTime = mutationBuffer.getLong(position); From bbf84b8d903d5831607b4ece2a0de509f5946afe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 13:03:15 +0200 Subject: [PATCH 19/45] Move first-last-string extension into druid core --- .../extensions-contrib/first-last-string.md | 41 ---------- extensions-contrib/first-last-string/pom.xml | 81 ------------------- .../FirstLastStringDruidModule.java | 60 -------------- .../io.druid.initialization.DruidModule | 1 - .../src/test/resources/sample.data.tsv | 13 --- pom.xml | 1 - .../io/druid/jackson/AggregatorsModule.java | 11 ++- .../SerializablePairLongString.java | 0 .../SerializablePairLongStringSerde.java | 0 .../first/StringFirstAggregateCombiner.java | 0 .../first/StringFirstAggregator.java | 0 .../first/StringFirstAggregatorFactory.java | 0 .../first/StringFirstBufferAggregator.java | 0 .../StringFirstFoldingAggregatorFactory.java | 0 .../last/StringLastAggregateCombiner.java | 0 .../last/StringLastAggregator.java | 0 .../last/StringLastAggregatorFactory.java | 0 .../last/StringLastBufferAggregator.java | 0 .../StringLastFoldingAggregatorFactory.java | 0 .../first/StringFirstAggregationTest.java | 4 - .../StringFirstBufferAggregatorTest.java | 0 .../first/StringFirstTimeseriesQueryTest.java | 9 --- .../last/StringLastAggregationTest.java | 4 - .../last/StringLastBufferAggregatorTest.java | 0 .../last/StringLastTimeseriesQueryTest.java | 9 --- 25 files changed, 10 insertions(+), 224 deletions(-) delete mode 100644 docs/content/development/extensions-contrib/first-last-string.md delete mode 100644 extensions-contrib/first-last-string/pom.xml delete mode 100644 extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java delete mode 100644 extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule delete mode 100644 extensions-contrib/first-last-string/src/test/resources/sample.data.tsv rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java (100%) rename {extensions-contrib/first-last-string => processing}/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java (100%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java (97%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java (100%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java (95%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java (97%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java (100%) rename {extensions-contrib/first-last-string => processing}/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java (95%) diff --git a/docs/content/development/extensions-contrib/first-last-string.md b/docs/content/development/extensions-contrib/first-last-string.md deleted file mode 100644 index e5a83ae86748..000000000000 --- a/docs/content/development/extensions-contrib/first-last-string.md +++ /dev/null @@ -1,41 +0,0 @@ ---- -layout: doc_page ---- - -# First/Last String Module - -To use these aggregators, make sure you [include](../../operations/including-extensions.html) the extension in your config file: - -``` -druid.extensions.loadList=["druid-first-last-string"] -``` - -## First String aggregator - -`stringFirst` computes the metric value with the minimum timestamp or `null` if no row exist - -```json -{ - "type" : "stringFirst", - "name" : , - "fieldName" : , - "maxStringBytes" : -} -``` - -## Last String aggregator - -```json -{ - "type" : "stringLast", - "name" : , - "fieldName" : , - "maxStringBytes" : -} -``` - -`stringLast` computes the metric value with the maximum timestamp or `null` if no row exist - - - -Note: The default value of `maxStringBytes` is 1024. diff --git a/extensions-contrib/first-last-string/pom.xml b/extensions-contrib/first-last-string/pom.xml deleted file mode 100644 index e7cc7a3f9551..000000000000 --- a/extensions-contrib/first-last-string/pom.xml +++ /dev/null @@ -1,81 +0,0 @@ - - - - 4.0.0 - - io.druid.extensions.contrib - druid-first-last-string - druid-first-last-string - druid-first-last-string - - - io.druid - druid - 0.13.0-SNAPSHOT - ../../pom.xml - - - - - io.druid - druid-processing - ${project.parent.version} - provided - - - io.druid - druid-sql - ${project.parent.version} - provided - - - - - io.druid - druid-processing - ${project.parent.version} - test - test-jar - - - io.druid - druid-sql - ${project.parent.version} - test-jar - test - - - io.druid - druid-server - ${project.parent.version} - test - test-jar - - - junit - junit - test - - - org.easymock - easymock - test - - - - diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java b/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java deleted file mode 100644 index 3587c5a7aa36..000000000000 --- a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/FirstLastStringDruidModule.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.databind.Module; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import io.druid.initialization.DruidModule; -import io.druid.query.aggregation.first.StringFirstAggregatorFactory; -import io.druid.query.aggregation.first.StringFirstFoldingAggregatorFactory; -import io.druid.query.aggregation.last.StringLastAggregatorFactory; -import io.druid.query.aggregation.last.StringLastFoldingAggregatorFactory; -import io.druid.segment.serde.ComplexMetrics; - -import java.util.List; - -/** - */ -public class FirstLastStringDruidModule implements DruidModule -{ - - @Override - public List getJacksonModules() - { - return ImmutableList.of( - new SimpleModule("FirstLastStringModule").registerSubtypes( - StringLastAggregatorFactory.class, - StringLastFoldingAggregatorFactory.class, - StringFirstAggregatorFactory.class, - StringFirstFoldingAggregatorFactory.class - ) - ); - } - - @Override - public void configure(Binder binder) - { - if (ComplexMetrics.getSerdeForType("serializablePairLongString") == null) { - ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairLongStringSerde()); - } - } -} diff --git a/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule deleted file mode 100644 index 981e7aee02fc..000000000000 --- a/extensions-contrib/first-last-string/src/main/resources/META-INF/services/io.druid.initialization.DruidModule +++ /dev/null @@ -1 +0,0 @@ -io.druid.query.aggregation.FirstLastStringDruidModule \ No newline at end of file diff --git a/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv b/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv deleted file mode 100644 index 674d86cefe9f..000000000000 --- a/extensions-contrib/first-last-string/src/test/resources/sample.data.tsv +++ /dev/null @@ -1,13 +0,0 @@ -2011-04-15T00:00:00.000Z spot automotive preferred apreferred 106.793700 -2011-04-15T00:00:00.000Z spot business preferred bpreferred 94.469747 -2011-04-15T00:00:00.000Z spot entertainment preferred epreferred 135.109191 -2011-04-15T00:00:00.000Z spot health preferred hpreferred 99.596909 -2011-04-15T00:00:00.000Z spot mezzanine preferred mpreferred 92.782760 -2011-04-15T00:00:00.000Z spot news preferred npreferred -2011-04-15T00:00:00.000Z spot premium preferred ppreferred -2011-04-15T00:00:00.000Z spot technology preferred tpreferred -2011-04-15T00:00:00.000Z spot travel preferred tpreferred -2011-04-15T00:00:00.000Z total_market mezzanine preferred mpreferred -2011-04-15T00:00:00.000Z total_market premium preferred ppreferred -2011-04-15T00:00:00.000Z upfront mezzanine preferred mpreferred -2011-04-15T00:00:00.000Z upfront premium preferred ppreferred diff --git a/pom.xml b/pom.xml index 165cea23e93a..0359fc52e648 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,6 @@ extensions-contrib/kafka-emitter extensions-contrib/redis-cache extensions-contrib/opentsdb-emitter - extensions-contrib/first-last-string extensions-contrib/materialized-view-maintenance extensions-contrib/materialized-view-selection diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index 3635b04b3e5f..c9d6b43e4c35 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -38,10 +38,12 @@ import io.druid.query.aggregation.LongMinAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.SerializablePairLongStringSerde; import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory; import io.druid.query.aggregation.first.DoubleFirstAggregatorFactory; import io.druid.query.aggregation.first.FloatFirstAggregatorFactory; import io.druid.query.aggregation.first.LongFirstAggregatorFactory; +import io.druid.query.aggregation.first.StringFirstAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; @@ -49,6 +51,7 @@ import io.druid.query.aggregation.last.DoubleLastAggregatorFactory; import io.druid.query.aggregation.last.FloatLastAggregatorFactory; import io.druid.query.aggregation.last.LongLastAggregatorFactory; +import io.druid.query.aggregation.last.StringLastAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.DoubleGreatestPostAggregator; @@ -77,6 +80,10 @@ public AggregatorsModule() ComplexMetrics.registerSerde("preComputedHyperUnique", new PreComputedHyperUniquesSerde(HyperLogLogHash.getDefault())); } + if (ComplexMetrics.getSerdeForType("serializablePairLongString") == null) { + ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairLongStringSerde()); + } + setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class); setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class); } @@ -101,9 +108,11 @@ public AggregatorsModule() @JsonSubTypes.Type(name = "longFirst", value = LongFirstAggregatorFactory.class), @JsonSubTypes.Type(name = "doubleFirst", value = DoubleFirstAggregatorFactory.class), @JsonSubTypes.Type(name = "floatFirst", value = FloatFirstAggregatorFactory.class), + @JsonSubTypes.Type(name = "stringFirst", value = StringFirstAggregatorFactory.class), @JsonSubTypes.Type(name = "longLast", value = LongLastAggregatorFactory.class), @JsonSubTypes.Type(name = "doubleLast", value = DoubleLastAggregatorFactory.class), - @JsonSubTypes.Type(name = "floatLast", value = FloatLastAggregatorFactory.class) + @JsonSubTypes.Type(name = "floatLast", value = FloatLastAggregatorFactory.class), + @JsonSubTypes.Type(name = "stringLast", value = StringLastAggregatorFactory.class), }) public interface AggregatorFactoryMixin { diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java rename to processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java rename to processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java rename to processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java rename to processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java rename to processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java rename to processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java rename to processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java rename to processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java diff --git a/extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java similarity index 100% rename from extensions-contrib/first-last-string/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java rename to processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java similarity index 97% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java rename to processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 58a8a4a2720b..1f4620e7e6dd 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -20,7 +20,6 @@ package io.druid.query.aggregation.first; import io.druid.java.util.common.Pair; -import io.druid.query.aggregation.FirstLastStringDruidModule; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; @@ -65,9 +64,6 @@ public void setup() EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.replay(colSelectorFactory); - - FirstLastStringDruidModule module = new FirstLastStringDruidModule(); - module.configure(null); } @Test diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java similarity index 100% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java rename to processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java similarity index 95% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java rename to processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index d33d6d797867..0bb372a78288 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -28,7 +28,6 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.query.aggregation.FirstLastStringDruidModule; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; @@ -38,7 +37,6 @@ import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.DateTime; -import org.junit.Before; import org.junit.Test; import java.util.Collections; @@ -47,13 +45,6 @@ public class StringFirstTimeseriesQueryTest { - @Before - public void setup() - { - FirstLastStringDruidModule module = new FirstLastStringDruidModule(); - module.configure(null); - } - @Test public void testTopNWithDistinctCountAgg() throws Exception { diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java similarity index 97% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java rename to processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index 01d32af17508..0f5c94718440 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -20,7 +20,6 @@ package io.druid.query.aggregation.last; import io.druid.java.util.common.Pair; -import io.druid.query.aggregation.FirstLastStringDruidModule; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; @@ -65,9 +64,6 @@ public void setup() EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.replay(colSelectorFactory); - - FirstLastStringDruidModule module = new FirstLastStringDruidModule(); - module.configure(null); } @Test diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java similarity index 100% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java rename to processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java diff --git a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java similarity index 95% rename from extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java rename to processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 339d2ab4e84c..acc759aa24c5 100644 --- a/extensions-contrib/first-last-string/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -28,7 +28,6 @@ import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.query.aggregation.FirstLastStringDruidModule; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; @@ -38,7 +37,6 @@ import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.joda.time.DateTime; -import org.junit.Before; import org.junit.Test; import java.util.Collections; @@ -47,13 +45,6 @@ public class StringLastTimeseriesQueryTest { - @Before - public void setup() - { - FirstLastStringDruidModule module = new FirstLastStringDruidModule(); - module.configure(null); - } - @Test public void testTopNWithDistinctCountAgg() throws Exception { From b09217c86bff669ebd910bce2a5f89c52dce4f3a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 13:03:35 +0200 Subject: [PATCH 20/45] Update druid core docs --- docs/content/development/extensions.md | 1 - docs/content/querying/aggregations.md | 30 +++++++++++++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index 96e31432d680..652901b2af38 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -72,7 +72,6 @@ All of these community extensions can be downloaded using *pull-deps* with the c |kafka-emitter|Kafka metrics emitter|[link](../development/extensions-contrib/kafka-emitter.html)| |druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| |druid-opentsdb-emitter|OpenTSDB metrics emitter |[link](../development/extensions-contrib/opentsdb-emitter.html)| -|druid-first-last-string|First/Last String Aggregators |[link](../development/extensions-contrib/first-last-string.html)| ## Promoting Community Extension to Core Extension diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md index b0ce5cc24c9d..50d7cdb00383 100644 --- a/docs/content/querying/aggregations.md +++ b/docs/content/querying/aggregations.md @@ -102,7 +102,7 @@ Computes and stores the sum of values as 32-bit floating point value. Similar to ### First / Last aggregator -First and Last aggregator cannot be used in ingestion spec, and should only be specified as part of queries. +(Double/Float/Long) First and Last aggregator cannot be used in ingestion spec, and should only be specified as part of queries. Note that queries with first/last aggregators on a segment created with rollup enabled will return the rolled up value, and not the last value within the raw ingested data. @@ -178,6 +178,34 @@ Note that queries with first/last aggregators on a segment created with rollup e } ``` +#### `stringFirst` aggregator + +`stringFirst` computes the metric value with the minimum timestamp or `null` if no row exist + +```json +{ + "type" : "stringFirst", + "name" : , + "fieldName" : , + "maxStringBytes" : # (optional, defaults to 1024) +} +``` + + + +#### `stringLast` aggregator + +`stringLast` computes the metric value with the maximum timestamp or `null` if no row exist + +```json +{ + "type" : "stringLast", + "name" : , + "fieldName" : , + "maxStringBytes" : # (optional, defaults to 1024) +} +``` + ### JavaScript aggregator Computes an arbitrary JavaScript function over a set of columns (both metrics and dimensions are allowed). Your From 834662d47c8c804b380bb060426f8a5c2da5d075 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 14:42:23 +0200 Subject: [PATCH 21/45] Fix null pointer exception when pair->string is null --- .../StringFirstFoldingAggregatorFactory.java | 24 +++++++++++-------- .../StringLastFoldingAggregatorFactory.java | 23 ++++++++++-------- 2 files changed, 27 insertions(+), 20 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index c1079752469a..8728b9bf392b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -70,18 +70,22 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) @Override public void aggregate(ByteBuffer buf, int position) { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - long lastTime = mutationBuffer.getLong(position); - if (pair != null && pair.lhs < lastTime) { - mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = StringUtils.toUtf8(pair.rhs); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + if (pair != null && pair.rhs != null) { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + long lastTime = mutationBuffer.getLong(position); + + if (pair.lhs < lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index c4b346e46516..2b495c00d627 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -70,18 +70,21 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) @Override public void aggregate(ByteBuffer buf, int position) { - ByteBuffer mutationBuffer = buf.duplicate(); - mutationBuffer.position(position); - SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - long lastTime = mutationBuffer.getLong(position); - if (pair != null && pair.lhs >= lastTime) { - mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = StringUtils.toUtf8(pair.rhs); + if (pair != null && pair.rhs != null) { + ByteBuffer mutationBuffer = buf.duplicate(); + mutationBuffer.position(position); + + long lastTime = mutationBuffer.getLong(position); + + if (pair.lhs >= lastTime) { + mutationBuffer.putLong(position, pair.lhs); + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } } } From 482e8bf0eaa1e078a2654012a9f840db312a6018 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 15:02:56 +0200 Subject: [PATCH 22/45] Add null control unit tests --- .../first/StringFirstAggregationTest.java | 7 +- .../StringFirstBufferAggregatorTest.java | 81 ++++++++++++++++++ .../last/StringLastAggregationTest.java | 7 +- .../last/StringLastBufferAggregatorTest.java | 83 +++++++++++++++++++ 4 files changed, 172 insertions(+), 6 deletions(-) diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 1f4620e7e6dd..f1588d2b81a9 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -42,13 +42,14 @@ public class StringFirstAggregationTest private TestObjectColumnSelector valueSelector; private TestObjectColumnSelector objectSelector; - private String[] strings = {"1111", "2222", "3333", "4444"}; - private long[] times = {8224, 6879, 2436, 7888}; + private String[] strings = {"1111", "2222", "3333", null, "4444"}; + private long[] times = {8224, 6879, 2436, 3546, 7888}; private SerializablePairLongString[] pairs = { new SerializablePairLongString(52782L, "AAAA"), new SerializablePairLongString(65492L, "BBBB"), new SerializablePairLongString(69134L, "CCCC"), - new SerializablePairLongString(11111L, "DDDD") + new SerializablePairLongString(11111L, "DDDD"), + new SerializablePairLongString(51223L, null) }; @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index 43a0adb76dd3..6ed916b65314 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -88,4 +88,85 @@ public void testBufferAggregate() throws Exception } + @Test + public void testNullBufferAggregate() throws Exception + { + + final long[] timestamps = {1526724000L, 1526724600L, 1526724700L, 1526725900L, 1526725000L}; + final String[] strings = {null, "AAAA", "BBBB", "DDDD", "EEEE"}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringFirstBufferAggregator agg = new StringFirstBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[1], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[1]), new Long(sp.lhs)); + + } + + @Test(expected = IllegalStateException.class) + public void testNoStringValue() + { + + final long[] timestamps = {1526724000L, 1526724600L}; + final Double[] doubles = {null, 2.00}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); + + StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringFirstBufferAggregator agg = new StringFirstBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + } + } diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index 0f5c94718440..f2462826f7c8 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -42,13 +42,14 @@ public class StringLastAggregationTest private TestObjectColumnSelector valueSelector; private TestObjectColumnSelector objectSelector; - private String[] strings = {"1111", "2222", "3333", "4444"}; - private long[] times = {8224, 6879, 2436, 7888}; + private String[] strings = {"1111", "2222", "3333", null, "4444"}; + private long[] times = {8224, 6879, 2436, 3546, 7888}; private SerializablePairLongString[] pairs = { new SerializablePairLongString(52782L, "AAAA"), new SerializablePairLongString(65492L, "BBBB"), new SerializablePairLongString(69134L, "CCCC"), - new SerializablePairLongString(11111L, "DDDD") + new SerializablePairLongString(11111L, "DDDD"), + new SerializablePairLongString(51223L, null) }; @Before diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index af82fb1208d0..0e1fd7ad6e37 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -23,6 +23,8 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.query.aggregation.first.StringFirstAggregatorFactory; +import io.druid.query.aggregation.first.StringFirstBufferAggregator; import org.junit.Assert; import org.junit.Test; @@ -88,4 +90,85 @@ public void testBufferAggregate() throws Exception } + @Test + public void testNullBufferAggregate() throws Exception + { + + final long[] timestamps = {1526724000L, 1526724600L, 1526724700L, 1526728900L, 1526725000L}; + final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[4], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[4]), new Long(sp.lhs)); + + } + + @Test(expected = IllegalStateException.class) + public void testNoStringValue() + { + + final long[] timestamps = {1526724000L, 1526724600L}; + final Double[] doubles = {null, 2.00}; + Integer maxStringBytes = 1024; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + } + } From 046c5a9bbe6757dd368de436e4ac06036e6516c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 15:06:26 +0200 Subject: [PATCH 23/45] Remove unused imports --- .../query/aggregation/last/StringLastBufferAggregatorTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index 0e1fd7ad6e37..e4cf6d53f048 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -23,8 +23,6 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; -import io.druid.query.aggregation.first.StringFirstAggregatorFactory; -import io.druid.query.aggregation.first.StringFirstBufferAggregator; import org.junit.Assert; import org.junit.Test; From 9829263e7344b551904ec9eba5aab732d121e311 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Sun, 10 Jun 2018 15:37:20 +0200 Subject: [PATCH 24/45] Add first/last string folding aggregator on AggregatorsModule to support segment metadata query --- .../main/java/io/druid/jackson/AggregatorsModule.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java index c9d6b43e4c35..22f1665746d9 100644 --- a/processing/src/main/java/io/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/io/druid/jackson/AggregatorsModule.java @@ -44,6 +44,7 @@ import io.druid.query.aggregation.first.FloatFirstAggregatorFactory; import io.druid.query.aggregation.first.LongFirstAggregatorFactory; import io.druid.query.aggregation.first.StringFirstAggregatorFactory; +import io.druid.query.aggregation.first.StringFirstFoldingAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; @@ -52,6 +53,7 @@ import io.druid.query.aggregation.last.FloatLastAggregatorFactory; import io.druid.query.aggregation.last.LongLastAggregatorFactory; import io.druid.query.aggregation.last.StringLastAggregatorFactory; +import io.druid.query.aggregation.last.StringLastFoldingAggregatorFactory; import io.druid.query.aggregation.post.ArithmeticPostAggregator; import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.aggregation.post.DoubleGreatestPostAggregator; @@ -77,7 +79,10 @@ public AggregatorsModule() } if (ComplexMetrics.getSerdeForType("preComputedHyperUnique") == null) { - ComplexMetrics.registerSerde("preComputedHyperUnique", new PreComputedHyperUniquesSerde(HyperLogLogHash.getDefault())); + ComplexMetrics.registerSerde( + "preComputedHyperUnique", + new PreComputedHyperUniquesSerde(HyperLogLogHash.getDefault()) + ); } if (ComplexMetrics.getSerdeForType("serializablePairLongString") == null) { @@ -109,10 +114,12 @@ public AggregatorsModule() @JsonSubTypes.Type(name = "doubleFirst", value = DoubleFirstAggregatorFactory.class), @JsonSubTypes.Type(name = "floatFirst", value = FloatFirstAggregatorFactory.class), @JsonSubTypes.Type(name = "stringFirst", value = StringFirstAggregatorFactory.class), + @JsonSubTypes.Type(name = "stringFirstFold", value = StringFirstFoldingAggregatorFactory.class), @JsonSubTypes.Type(name = "longLast", value = LongLastAggregatorFactory.class), @JsonSubTypes.Type(name = "doubleLast", value = DoubleLastAggregatorFactory.class), @JsonSubTypes.Type(name = "floatLast", value = FloatLastAggregatorFactory.class), @JsonSubTypes.Type(name = "stringLast", value = StringLastAggregatorFactory.class), + @JsonSubTypes.Type(name = "stringLastFold", value = StringLastFoldingAggregatorFactory.class) }) public interface AggregatorFactoryMixin { From 023fc88232c674df88367658912a452bcb15ef49 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:50:20 +0200 Subject: [PATCH 25/45] Change SerializablePairLongString to extend SerializablePair --- .../aggregation/SerializablePairLongString.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java index 20eb1dad9ac6..1fba28382178 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java @@ -21,25 +21,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.java.util.common.Pair; +import io.druid.collections.SerializablePair; -public class SerializablePairLongString extends Pair +public class SerializablePairLongString extends SerializablePair { @JsonCreator public SerializablePairLongString(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") String rhs) { super(lhs, rhs); } +} - @JsonProperty - public Long getLhs() - { - return lhs; - } - @JsonProperty - public String getRhs() - { - return rhs; - } -} From e6dee785152c8a098e1ec8d9fc17dea1b5301ef4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:53:50 +0200 Subject: [PATCH 26/45] Change vars from public to private --- .../query/aggregation/SerializablePairLongStringSerde.java | 2 +- .../query/aggregation/last/StringLastAggregateCombiner.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 2aa29fd2c464..cd451b9314df 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -43,7 +43,7 @@ public class SerializablePairLongStringSerde extends ComplexMetricSerde { - public static final String TYPE_NAME = "serializablePairLongString"; + private static final String TYPE_NAME = "serializablePairLongString"; @Override public String getTypeName() diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java index 19cc6dd6d3ec..4e29daa98854 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java @@ -26,7 +26,7 @@ public class StringLastAggregateCombiner extends ObjectAggregateCombiner { - String lastString; + private String lastString; @Override public void reset(ColumnValueSelector selector) From 1fb67897f9fdb175ce974c608e815ba79eb10bac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:55:18 +0200 Subject: [PATCH 27/45] Convert vars to primitive type --- .../query/aggregation/SerializablePairLongStringSerde.java | 4 ++-- .../druid/query/aggregation/first/StringFirstAggregator.java | 4 ++-- .../io/druid/query/aggregation/last/StringLastAggregator.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index cd451b9314df..fd5b2f5967cc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -132,8 +132,8 @@ public SerializablePairLongString fromByteBuffer(ByteBuffer buffer, int numBytes { final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); - Long lhs = readOnlyBuffer.getLong(); - Integer stringSize = readOnlyBuffer.getInt(); + long lhs = readOnlyBuffer.getLong(); + int stringSize = readOnlyBuffer.getInt(); String lastString = null; if (stringSize > 0) { diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index e20b976f76a4..cfc301abaf57 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -29,7 +29,7 @@ public class StringFirstAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; - private final Integer maxStringBytes; + private final int maxStringBytes; protected long firstTime; protected String firstValue; @@ -37,7 +37,7 @@ public class StringFirstAggregator implements Aggregator public StringFirstAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - Integer maxStringBytes + int maxStringBytes ) { this.valueSelector = valueSelector; diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index 81b90143990d..d7a227e6ede2 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -29,7 +29,7 @@ public class StringLastAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; - private final Integer maxStringBytes; + private final int maxStringBytes; protected long lastTime; protected String lastValue; @@ -37,7 +37,7 @@ public class StringLastAggregator implements Aggregator public StringLastAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - Integer maxStringBytes + int maxStringBytes ) { this.valueSelector = valueSelector; From 42b6ca3862ddd0199dc5d0432de4fb2a7a297091 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:55:51 +0200 Subject: [PATCH 28/45] Clarify compare comment --- .../query/aggregation/SerializablePairLongStringSerde.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index fd5b2f5967cc..6b3c4d93e409 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -112,7 +112,9 @@ public int compare(SerializablePairLongString o1, SerializablePairLongString o2) } else { // If the strings are not null, we will compare them - // Note: This comparation maybe doesn't make sense to first/last aggregators + // Note: This comparison maybe doesn't make sense to first/last aggregators, + // because compare both SerializablePairLongString based on String maybe doesn't matter + // to define each is first or last. comparation = o1.rhs.compareTo(o2.rhs); } } From a9f2d61bb34bb7ab0960198c1fcb7c3d03713610 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:57:14 +0200 Subject: [PATCH 29/45] Change IllegalStateException to ISE --- .../query/aggregation/first/StringFirstAggregator.java | 8 ++++---- .../aggregation/first/StringFirstBufferAggregator.java | 8 ++++---- .../query/aggregation/last/StringLastAggregator.java | 8 ++++---- .../aggregation/last/StringLastBufferAggregator.java | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index cfc301abaf57..da14e1414ed5 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation.first; +import io.druid.java.util.common.ISE; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.segment.BaseLongColumnValueSelector; @@ -61,10 +62,9 @@ public void aggregate() } else if (value instanceof SerializablePairLongString) { firstValue = ((SerializablePairLongString) value).rhs; } else if (value != null) { - throw new IllegalStateException( - "Try to aggregate unsuported class type [" - + value.getClass().getName() + - "]. Supported class types: String or SerializablePairLongString" + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() ); } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index 47bd0d437796..dcbab148471d 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation.first; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.SerializablePairLongString; @@ -72,10 +73,9 @@ public void aggregate(ByteBuffer buf, int position) time = timeSelector.getLong(); firstString = (String) value; } else if (value != null) { - throw new IllegalStateException( - "Try to aggregate unsuported class type [" - + value.getClass().getName() + - "]. Supported class types: String or SerializablePairLongString" + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() ); } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index d7a227e6ede2..c302637b6a16 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation.last; +import io.druid.java.util.common.ISE; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.SerializablePairLongString; import io.druid.segment.BaseLongColumnValueSelector; @@ -61,10 +62,9 @@ public void aggregate() } else if (value instanceof SerializablePairLongString) { lastValue = ((SerializablePairLongString) value).rhs; } else if (value != null) { - throw new IllegalStateException( - "Try to aggregate unsuported class type [" - + value.getClass().getName() + - "]. Supported class types: String or SerializablePairLongString" + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() ); } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index aabe04634295..52d7ba25e1c0 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -19,6 +19,7 @@ package io.druid.query.aggregation.last; +import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.SerializablePairLongString; @@ -72,10 +73,9 @@ public void aggregate(ByteBuffer buf, int position) time = timeSelector.getLong(); lastString = (String) value; } else if (value != null) { - throw new IllegalStateException( - "Try to aggregate unsuported class type [" - + value.getClass().getName() + - "]. Supported class types: String or SerializablePairLongString" + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() ); } From 5a1643f738c11c21e511a6d15297eebcfe18a4e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:57:40 +0200 Subject: [PATCH 30/45] Remove TODO comments --- .../query/aggregation/first/StringFirstBufferAggregator.java | 1 - .../druid/query/aggregation/last/StringLastBufferAggregator.java | 1 - 2 files changed, 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index dcbab148471d..df04dc0d3041 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; -// TODO: Unit Test public class StringFirstBufferAggregator implements BufferAggregator { private final BaseLongColumnValueSelector timeSelector; diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 52d7ba25e1c0..f6e5bd3c7e81 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -29,7 +29,6 @@ import java.nio.ByteBuffer; -//TODO: Unit Test public class StringLastBufferAggregator implements BufferAggregator { private final BaseLongColumnValueSelector timeSelector; From a9a1069e345a7af703033d0090685437082d86ed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 11 Jun 2018 10:58:04 +0200 Subject: [PATCH 31/45] Control possible null pointer exception --- .../aggregation/first/StringFirstFoldingAggregatorFactory.java | 2 +- .../aggregation/last/StringLastFoldingAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index 8728b9bf392b..3b2b3bdabc5b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -72,7 +72,7 @@ public void aggregate(ByteBuffer buf, int position) { SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - if (pair != null && pair.rhs != null) { + if (pair != null && pair.rhs != null && pair.lhs != null) { ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index 2b495c00d627..4b2854958fe6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -71,7 +71,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public void aggregate(ByteBuffer buf, int position) { SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - if (pair != null && pair.rhs != null) { + if (pair != null && pair.rhs != null && pair.lhs != null) { ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); From 1e53094c9283d96af2d337874745952dba67fe77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 09:00:03 +0200 Subject: [PATCH 32/45] Add @Nullable annotation --- .../query/aggregation/SerializablePairLongStringSerde.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 6b3c4d93e409..6c135cd31bc9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -31,6 +31,7 @@ import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; import io.druid.segment.writeout.SegmentWriteOutMedium; +import javax.annotation.Nullable; import java.nio.ByteBuffer; /** @@ -83,7 +84,7 @@ public ObjectStrategy getObjectStrategy() return new ObjectStrategy() { @Override - public int compare(SerializablePairLongString o1, SerializablePairLongString o2) + public int compare(@Nullable SerializablePairLongString o1, @Nullable SerializablePairLongString o2) { int comparation; From 129d5943441ae3aab0378c97139707a17f0b799d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 09:00:50 +0200 Subject: [PATCH 33/45] Remove empty line --- .../druid/query/aggregation/SerializablePairLongStringSerde.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 6c135cd31bc9..f63d2d2e70f1 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -154,7 +154,6 @@ public byte[] toBytes(SerializablePairLongString val) String rhsString = val.rhs; ByteBuffer bbuf; - if (rhsString != null) { byte[] rhsBytes = StringUtils.toUtf8(rhsString); bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsBytes.length); From a5dea442c2b3661785c6bcb5b1977538fb653566 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 09:03:03 +0200 Subject: [PATCH 34/45] Remove unused parameter type --- .../query/aggregation/first/StringFirstAggregatorFactory.java | 2 +- .../query/aggregation/last/StringLastAggregatorFactory.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 7d7d77307483..6fc31ab40fcf 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -129,7 +129,7 @@ public AggregatorFactory getCombiningFactory() @Override public List getRequiredColumns() { - return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index 853037ab7cc1..fe488ff07130 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -109,7 +109,7 @@ public AggregatorFactory getCombiningFactory() @Override public List getRequiredColumns() { - return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override From e7992fac2b8bfd5c33fddd2ff954e8bd88f854e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 09:25:15 +0200 Subject: [PATCH 35/45] Improve AggregatorCombiner javadocs --- .../main/java/io/druid/query/aggregation/AggregateCombiner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java index 797f44843837..6a971db40fc0 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java @@ -63,7 +63,7 @@ public interface AggregateCombiner extends ColumnValueSelector * become a subject for modification during subsequent fold() calls. * * Since the state of AggregateCombiner is undefined before {@link #reset} is ever called on it, the effects of - * calling fold() are also undefined in this case. + * calling fold() are also undefined in this case. The {@link #reset} is called first before {@link #fold} * * @see AggregatorFactory#combine */ From aa4ff30c76de222cf18c9da4c61a2766ab135038 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 10:34:36 +0200 Subject: [PATCH 36/45] Add filterNullValues option at StringLast and StringFirst aggregators --- .../first/StringFirstAggregator.java | 33 +++-- .../first/StringFirstAggregatorFactory.java | 28 +++-- .../first/StringFirstBufferAggregator.java | 56 +++++---- .../StringFirstFoldingAggregatorFactory.java | 24 ++-- .../last/StringLastAggregator.java | 33 +++-- .../last/StringLastAggregatorFactory.java | 32 +++-- .../last/StringLastBufferAggregator.java | 56 +++++---- .../StringLastFoldingAggregatorFactory.java | 23 ++-- .../first/StringFirstAggregationTest.java | 2 +- .../StringFirstBufferAggregatorTest.java | 111 +++++++++++++++-- .../first/StringFirstTimeseriesQueryTest.java | 6 +- .../last/StringLastAggregationTest.java | 2 +- .../last/StringLastBufferAggregatorTest.java | 113 ++++++++++++++++-- .../last/StringLastTimeseriesQueryTest.java | 11 +- 14 files changed, 402 insertions(+), 128 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index da14e1414ed5..03421e8d0924 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -31,6 +31,7 @@ public class StringFirstAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; private final int maxStringBytes; + private final boolean filterNullValues; protected long firstTime; protected String firstValue; @@ -38,12 +39,14 @@ public class StringFirstAggregator implements Aggregator public StringFirstAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes + int maxStringBytes, + boolean filterNullValues ) { this.valueSelector = valueSelector; this.timeSelector = timeSelector; this.maxStringBytes = maxStringBytes; + this.filterNullValues = filterNullValues; firstTime = Long.MAX_VALUE; firstValue = null; @@ -57,19 +60,23 @@ public void aggregate() firstTime = time; Object value = valueSelector.getObject(); - if (value instanceof String) { - firstValue = (String) value; - } else if (value instanceof SerializablePairLongString) { - firstValue = ((SerializablePairLongString) value).rhs; - } else if (value != null) { - throw new ISE( - "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", - value.getClass().getCanonicalName() - ); - } + if (value != null) { + if (value instanceof String) { + firstValue = (String) value; + } else if (value instanceof SerializablePairLongString) { + firstValue = ((SerializablePairLongString) value).rhs; + } else { + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() + ); + } - if (firstValue != null && firstValue.length() > maxStringBytes) { - firstValue = firstValue.substring(0, maxStringBytes); + if (firstValue != null && firstValue.length() > maxStringBytes) { + firstValue = firstValue.substring(0, maxStringBytes); + } + } else if (!filterNullValues) { + firstValue = null; } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 6fc31ab40fcf..34244eb3d1f7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -67,12 +67,14 @@ public class StringFirstAggregatorFactory extends AggregatorFactory private final String fieldName; private final String name; protected final int maxStringBytes; + protected final boolean filterNullValues; @JsonCreator public StringFirstAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes + @JsonProperty("maxStringBytes") Integer maxStringBytes, + @JsonProperty("filterNullValues") Boolean filterNullValues ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); @@ -80,6 +82,7 @@ public StringFirstAggregatorFactory( this.name = name; this.fieldName = fieldName; this.maxStringBytes = maxStringBytes == null ? DEFAULT_MAX_STRING_SIZE : maxStringBytes; + this.filterNullValues = filterNullValues == null ? false : filterNullValues; } @Override @@ -88,7 +91,8 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) return new StringFirstAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes + maxStringBytes, + filterNullValues ); } @@ -98,7 +102,8 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) return new StringFirstBufferAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes + maxStringBytes, + filterNullValues ); } @@ -123,13 +128,13 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringFirstFoldingAggregatorFactory(name, name, maxStringBytes); + return new StringFirstFoldingAggregatorFactory(name, name, maxStringBytes, filterNullValues); } @Override public List getRequiredColumns() { - return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes, filterNullValues)); } @Override @@ -164,6 +169,12 @@ public Integer getMaxStringBytes() return maxStringBytes; } + @JsonProperty + public Boolean getFilterNullValues() + { + return filterNullValues; + } + @Override public List requiredFields() { @@ -176,6 +187,7 @@ public byte[] getCacheKey() return new CacheKeyBuilder(AggregatorUtil.STRING_FIRST_CACHE_TYPE_ID) .appendString(fieldName) .appendInt(maxStringBytes) + .appendBoolean(filterNullValues) .build(); } @@ -203,13 +215,14 @@ public boolean equals(Object o) StringFirstAggregatorFactory that = (StringFirstAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; + return fieldName.equals(that.fieldName) && name.equals(that.name) && + maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; } @Override public int hashCode() { - return Objects.hash(name, fieldName, maxStringBytes); + return Objects.hash(name, fieldName, maxStringBytes, filterNullValues); } @Override @@ -219,6 +232,7 @@ public String toString() "name='" + name + '\'' + ", fieldName='" + fieldName + '\'' + ", maxStringBytes=" + maxStringBytes + '\'' + + ", filterNullValues=" + filterNullValues + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index df04dc0d3041..20b2725692fc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -34,16 +34,19 @@ public class StringFirstBufferAggregator implements BufferAggregator private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; private final int maxStringBytes; + private final boolean filterNullValues; public StringFirstBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes + int maxStringBytes, + boolean filterNullValues ) { this.timeSelector = timeSelector; this.valueSelector = valueSelector; this.maxStringBytes = maxStringBytes; + this.filterNullValues = filterNullValues; } @Override @@ -61,36 +64,43 @@ public void aggregate(ByteBuffer buf, int position) Object value = valueSelector.getObject(); - long time = Long.MAX_VALUE; + long time = timeSelector.getLong(); String firstString = null; - if (value instanceof SerializablePairLongString) { - SerializablePairLongString serializablePair = (SerializablePairLongString) value; - time = serializablePair.lhs; - firstString = serializablePair.rhs; - } else if (value instanceof String) { - time = timeSelector.getLong(); - firstString = (String) value; - } else if (value != null) { - throw new ISE( - "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", - value.getClass().getCanonicalName() - ); + if (value != null) { + if (value instanceof SerializablePairLongString) { + SerializablePairLongString serializablePair = (SerializablePairLongString) value; + time = serializablePair.lhs; + firstString = serializablePair.rhs; + } else if (value instanceof String) { + firstString = (String) value; + } else { + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() + ); + } } long lastTime = mutationBuffer.getLong(position); - if (firstString != null && time < lastTime) { - if (firstString.length() > maxStringBytes) { - firstString = firstString.substring(0, maxStringBytes); - } - byte[] valueBytes = StringUtils.toUtf8(firstString); + if (time < lastTime) { + if (firstString != null) { + if (firstString.length() > maxStringBytes) { + firstString = firstString.substring(0, maxStringBytes); + } - mutationBuffer.putLong(position, time); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + byte[] valueBytes = StringUtils.toUtf8(firstString); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } else if (!filterNullValues) { + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, 0); + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index 3b2b3bdabc5b..d52725a02777 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -37,17 +37,18 @@ public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFa public StringFirstFoldingAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes + @JsonProperty("maxStringBytes") Integer maxStringBytes, + @JsonProperty("filterNullValues") Boolean filterNullValues ) { - super(name, fieldName, maxStringBytes); + super(name, fieldName, maxStringBytes, filterNullValues); } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringFirstAggregator(null, null, maxStringBytes) + return new StringFirstAggregator(null, null, maxStringBytes, filterNullValues) { @Override public void aggregate() @@ -65,14 +66,14 @@ public void aggregate() public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringFirstBufferAggregator(null, null, maxStringBytes) + return new StringFirstBufferAggregator(null, null, maxStringBytes, filterNullValues) { @Override public void aggregate(ByteBuffer buf, int position) { SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - if (pair != null && pair.rhs != null && pair.lhs != null) { + if (pair != null && pair.lhs != null) { ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); @@ -80,11 +81,16 @@ public void aggregate(ByteBuffer buf, int position) if (pair.lhs < lastTime) { mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = StringUtils.toUtf8(pair.rhs); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + if (pair.rhs != null) { + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); + + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } else if (!filterNullValues) { + mutationBuffer.putInt(position + Long.BYTES, 0); + } } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index c302637b6a16..713699d61885 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -31,6 +31,7 @@ public class StringLastAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; private final int maxStringBytes; + private final boolean filterNullValues; protected long lastTime; protected String lastValue; @@ -38,12 +39,14 @@ public class StringLastAggregator implements Aggregator public StringLastAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes + int maxStringBytes, + boolean filterNullValues ) { this.valueSelector = valueSelector; this.timeSelector = timeSelector; this.maxStringBytes = maxStringBytes; + this.filterNullValues = filterNullValues; lastTime = Long.MIN_VALUE; lastValue = null; @@ -57,19 +60,23 @@ public void aggregate() lastTime = time; Object value = valueSelector.getObject(); - if (value instanceof String) { - lastValue = (String) value; - } else if (value instanceof SerializablePairLongString) { - lastValue = ((SerializablePairLongString) value).rhs; - } else if (value != null) { - throw new ISE( - "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", - value.getClass().getCanonicalName() - ); - } + if (value != null) { + if (value instanceof String) { + lastValue = (String) value; + } else if (value instanceof SerializablePairLongString) { + lastValue = ((SerializablePairLongString) value).rhs; + } else { + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() + ); + } - if (lastValue != null && lastValue.length() > maxStringBytes) { - lastValue = lastValue.substring(0, maxStringBytes); + if (lastValue != null && lastValue.length() > maxStringBytes) { + lastValue = lastValue.substring(0, maxStringBytes); + } + } else if (!filterNullValues) { + lastValue = null; } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index fe488ff07130..60090280b8ed 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -47,19 +47,24 @@ public class StringLastAggregatorFactory extends AggregatorFactory private final String fieldName; private final String name; protected final int maxStringBytes; + protected final boolean filterNullValues; @JsonCreator public StringLastAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes + @JsonProperty("maxStringBytes") Integer maxStringBytes, + @JsonProperty("filterNullValues") Boolean filterNullValues ) { 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; - this.maxStringBytes = maxStringBytes == null ? StringFirstAggregatorFactory.DEFAULT_MAX_STRING_SIZE : maxStringBytes; + this.maxStringBytes = maxStringBytes == null + ? StringFirstAggregatorFactory.DEFAULT_MAX_STRING_SIZE + : maxStringBytes; + this.filterNullValues = filterNullValues == null ? false : filterNullValues; } @Override @@ -68,7 +73,8 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) return new StringLastAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes + maxStringBytes, + filterNullValues ); } @@ -78,7 +84,8 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) return new StringLastBufferAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes + maxStringBytes, + filterNullValues ); } @@ -103,13 +110,13 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringLastFoldingAggregatorFactory(name, name, maxStringBytes); + return new StringLastFoldingAggregatorFactory(name, name, maxStringBytes, filterNullValues); } @Override public List getRequiredColumns() { - return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes, filterNullValues)); } @Override @@ -144,6 +151,12 @@ public Integer getMaxStringBytes() return maxStringBytes; } + @JsonProperty + public Boolean getFilterNullValues() + { + return filterNullValues; + } + @Override public List requiredFields() { @@ -156,6 +169,7 @@ public byte[] getCacheKey() return new CacheKeyBuilder(AggregatorUtil.STRING_LAST_CACHE_TYPE_ID) .appendString(fieldName) .appendInt(maxStringBytes) + .appendBoolean(filterNullValues) .build(); } @@ -183,13 +197,14 @@ public boolean equals(Object o) StringLastAggregatorFactory that = (StringLastAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; + return fieldName.equals(that.fieldName) && name.equals(that.name) && + maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; } @Override public int hashCode() { - return Objects.hash(name, fieldName, maxStringBytes); + return Objects.hash(name, fieldName, maxStringBytes, filterNullValues); } @Override @@ -199,6 +214,7 @@ public String toString() "name='" + name + '\'' + ", fieldName='" + fieldName + '\'' + ", maxStringBytes=" + maxStringBytes + '\'' + + ", filterNullValues=" + filterNullValues + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index f6e5bd3c7e81..94c74db20ca3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -34,16 +34,19 @@ public class StringLastBufferAggregator implements BufferAggregator private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; private final int maxStringBytes; + private final boolean filterNullValues; public StringLastBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes + int maxStringBytes, + boolean filterNullValues ) { this.timeSelector = timeSelector; this.valueSelector = valueSelector; this.maxStringBytes = maxStringBytes; + this.filterNullValues = filterNullValues; } @Override @@ -61,36 +64,43 @@ public void aggregate(ByteBuffer buf, int position) Object value = valueSelector.getObject(); - long time = Long.MIN_VALUE; + long time = timeSelector.getLong(); String lastString = null; - if (value instanceof SerializablePairLongString) { - SerializablePairLongString serializablePair = (SerializablePairLongString) value; - time = serializablePair.lhs; - lastString = serializablePair.rhs; - } else if (value instanceof String) { - time = timeSelector.getLong(); - lastString = (String) value; - } else if (value != null) { - throw new ISE( - "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", - value.getClass().getCanonicalName() - ); + if (value != null) { + if (value instanceof SerializablePairLongString) { + SerializablePairLongString serializablePair = (SerializablePairLongString) value; + time = serializablePair.lhs; + lastString = serializablePair.rhs; + } else if (value instanceof String) { + lastString = (String) value; + } else { + throw new ISE( + "Try to aggregate unsuported class type [%s].Supported class types: String or SerializablePairLongString", + value.getClass().getCanonicalName() + ); + } } long lastTime = mutationBuffer.getLong(position); - if (lastString != null && time >= lastTime) { - if (lastString.length() > maxStringBytes) { - lastString = lastString.substring(0, maxStringBytes); - } - byte[] valueBytes = StringUtils.toUtf8(lastString); + if (time >= lastTime) { + if (lastString != null) { + if (lastString.length() > maxStringBytes) { + lastString = lastString.substring(0, maxStringBytes); + } - mutationBuffer.putLong(position, time); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + byte[] valueBytes = StringUtils.toUtf8(lastString); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } else if (!filterNullValues) { + mutationBuffer.putLong(position, time); + mutationBuffer.putInt(position + Long.BYTES, 0); + } } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index 4b2854958fe6..f41ff0e3c46f 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -37,17 +37,18 @@ public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFact public StringLastFoldingAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes + @JsonProperty("maxStringBytes") Integer maxStringBytes, + @JsonProperty("filterNullValues") Boolean filterNullValues ) { - super(name, fieldName, maxStringBytes); + super(name, fieldName, maxStringBytes, filterNullValues); } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringLastAggregator(null, null, maxStringBytes) + return new StringLastAggregator(null, null, maxStringBytes, filterNullValues) { @Override public void aggregate() @@ -65,13 +66,13 @@ public void aggregate() public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringLastBufferAggregator(null, null, maxStringBytes) + return new StringLastBufferAggregator(null, null, maxStringBytes, filterNullValues) { @Override public void aggregate(ByteBuffer buf, int position) { SerializablePairLongString pair = (SerializablePairLongString) selector.getObject(); - if (pair != null && pair.rhs != null && pair.lhs != null) { + if (pair != null && pair.lhs != null) { ByteBuffer mutationBuffer = buf.duplicate(); mutationBuffer.position(position); @@ -79,11 +80,15 @@ public void aggregate(ByteBuffer buf, int position) if (pair.lhs >= lastTime) { mutationBuffer.putLong(position, pair.lhs); - byte[] valueBytes = StringUtils.toUtf8(pair.rhs); + if (pair.rhs != null) { + byte[] valueBytes = StringUtils.toUtf8(pair.rhs); - mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); - mutationBuffer.position(position + Long.BYTES + Integer.BYTES); - mutationBuffer.put(valueBytes); + mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); + mutationBuffer.position(position + Long.BYTES + Integer.BYTES); + mutationBuffer.put(valueBytes); + } else if (!filterNullValues) { + mutationBuffer.putInt(position + Long.BYTES, 0); + } } } } diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index f1588d2b81a9..1be3b3c6cffc 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -55,7 +55,7 @@ public class StringFirstAggregationTest @Before public void setup() { - stringLastAggFactory = new StringFirstAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); + stringLastAggFactory = new StringFirstAggregatorFactory("billy", "nilly", MAX_STRING_SIZE, false); combiningAggFactory = (StringFirstAggregatorFactory) stringLastAggFactory.getCombiningFactory(); timeSelector = new TestLongColumnSelector(times); valueSelector = new TestObjectColumnSelector<>(strings); diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index 6ed916b65314..45de88e63bae 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -23,6 +23,8 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; +import io.druid.query.aggregation.last.StringLastAggregatorFactory; +import io.druid.query.aggregation.last.StringLastBufferAggregator; import org.junit.Assert; import org.junit.Test; @@ -51,18 +53,20 @@ public void testBufferAggregate() throws Exception final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -92,21 +96,23 @@ public void testBufferAggregate() throws Exception public void testNullBufferAggregate() throws Exception { - final long[] timestamps = {1526724000L, 1526724600L, 1526724700L, 1526725900L, 1526725000L}; + final long[] timestamps = {2222L, 1111L, 3333L, 4444L, 5555L}; final String[] strings = {null, "AAAA", "BBBB", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -139,18 +145,20 @@ public void testNoStringValue() final long[] timestamps = {1526724000L, 1526724600L}; final Double[] doubles = {null, 2.00}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -169,4 +177,93 @@ public void testNoStringValue() } } + @Test + public void testFilterNullValues() throws Exception + { + + final long[] timestamps = {1111L, 2222L, 3333L, 1110L, 5555L}; + final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; + Integer maxStringBytes = 1024; + Boolean filterNullValues = true; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( + "billy", "billy", maxStringBytes, filterNullValues + ); + + StringFirstBufferAggregator agg = new StringFirstBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes, + filterNullValues + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[0], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[0]), new Long(sp.lhs)); + } + + @Test + public void testNotFilterNullValues() throws Exception + { + + final long[] timestamps = {1111L, 2222L, 3333L, 1110L, 5555L}; + final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; + Integer maxStringBytes = 1024; + Boolean filterNullValues = false; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( + "billy", "billy", maxStringBytes, filterNullValues + ); + + StringFirstBufferAggregator agg = new StringFirstBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes, + filterNullValues + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[3], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[3]), new Long(sp.lhs)); + } } diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index 0bb372a78288..f6421e662ec0 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -59,7 +59,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .withQueryGranularity(Granularities.SECOND) .withMetrics(new CountAggregatorFactory("cnt")) .withMetrics(new StringFirstAggregatorFactory( - "last_client_type", "client_type", 1024) + "last_client_type", "client_type", 1024, false) ) .build() ) @@ -100,7 +100,9 @@ public void testTopNWithDistinctCountAgg() throws Exception .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Lists.newArrayList( - new StringFirstAggregatorFactory("last_client_type", client_type, 1024) + new StringFirstAggregatorFactory( + "last_client_type", client_type, 1024, false + ) ) ) .build(); diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index f2462826f7c8..6a31a51a0f63 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -55,7 +55,7 @@ public class StringLastAggregationTest @Before public void setup() { - stringLastAggFactory = new StringLastAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); + stringLastAggFactory = new StringLastAggregatorFactory("billy", "nilly", MAX_STRING_SIZE, false); combiningAggFactory = (StringLastAggregatorFactory) stringLastAggFactory.getCombiningFactory(); timeSelector = new TestLongColumnSelector(times); valueSelector = new TestObjectColumnSelector<>(strings); diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index e4cf6d53f048..d83426903fba 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -51,18 +51,20 @@ public void testBufferAggregate() throws Exception final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -92,21 +94,23 @@ public void testBufferAggregate() throws Exception public void testNullBufferAggregate() throws Exception { - final long[] timestamps = {1526724000L, 1526724600L, 1526724700L, 1526728900L, 1526725000L}; + final long[] timestamps = {1111L, 2222L, 6666L, 4444L, 5555L}; final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -127,8 +131,8 @@ public void testNullBufferAggregate() throws Exception SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - Assert.assertEquals("expectec last string value", strings[4], sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[4]), new Long(sp.lhs)); + Assert.assertEquals("expectec last string value", strings[2], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[2]), new Long(sp.lhs)); } @@ -139,18 +143,20 @@ public void testNoStringValue() final long[] timestamps = {1526724000L, 1526724600L}; final Double[] doubles = {null, 2.00}; Integer maxStringBytes = 1024; + Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes + "billy", "billy", maxStringBytes, filterNullValues ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes + maxStringBytes, + filterNullValues ); String testString = "ZZZZ"; @@ -169,4 +175,93 @@ public void testNoStringValue() } } + @Test + public void testFilterNullValues() throws Exception + { + + final long[] timestamps = {1111L, 2222L, 3333L, 6666L, 5555L}; + final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; + Integer maxStringBytes = 1024; + Boolean filterNullValues = true; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes, filterNullValues + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes, + filterNullValues + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[4], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[4]), new Long(sp.lhs)); + } + + @Test + public void testNotFilterNullValues() throws Exception + { + + final long[] timestamps = {1111L, 2222L, 3333L, 6666L, 5555L}; + final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; + Integer maxStringBytes = 1024; + Boolean filterNullValues = false; + + TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); + TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); + + StringLastAggregatorFactory factory = new StringLastAggregatorFactory( + "billy", "billy", maxStringBytes, filterNullValues + ); + + StringLastBufferAggregator agg = new StringLastBufferAggregator( + longColumnSelector, + objectColumnSelector, + maxStringBytes, + filterNullValues + ); + + String testString = "ZZZZ"; + + ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); + buf.putLong(1526728500L); + buf.putInt(testString.length()); + buf.put(testString.getBytes(StandardCharsets.UTF_8)); + + int position = 0; + + agg.init(buf, position); + //noinspection ForLoopReplaceableByForEach + for (int i = 0; i < timestamps.length; i++) { + aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); + } + + SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); + + + Assert.assertEquals("expectec last string value", strings[3], sp.rhs); + Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[3]), new Long(sp.lhs)); + } } diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index acc759aa24c5..45c372ab627b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -59,7 +59,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .withQueryGranularity(Granularities.SECOND) .withMetrics(new CountAggregatorFactory("cnt")) .withMetrics(new StringLastAggregatorFactory( - "last_client_type", "client_type", 1024) + "last_client_type", "client_type", 1024, false) ) .build() ) @@ -100,7 +100,9 @@ public void testTopNWithDistinctCountAgg() throws Exception .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Lists.newArrayList( - new StringLastAggregatorFactory("last_client_type", client_type, 1024) + new StringLastAggregatorFactory( + "last_client_type", client_type, 1024, false + ) ) ) .build(); @@ -112,7 +114,10 @@ public void testTopNWithDistinctCountAgg() throws Exception new Result<>( time, new TimeseriesResultValue( - ImmutableMap.of("last_client_type", new SerializablePairLongString(timestamp1, "android")) + ImmutableMap.of( + "last_client_type", + new SerializablePairLongString(timestamp1, "android") + ) ) ) ); From 1d11d468192679e8d700e4176245756704e8f6d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 10:36:09 +0200 Subject: [PATCH 37/45] Add filterNullValues option at agg documentation --- docs/content/querying/aggregations.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/content/querying/aggregations.md b/docs/content/querying/aggregations.md index 50d7cdb00383..3f6b5e7c0968 100644 --- a/docs/content/querying/aggregations.md +++ b/docs/content/querying/aggregations.md @@ -187,7 +187,8 @@ Note that queries with first/last aggregators on a segment created with rollup e "type" : "stringFirst", "name" : , "fieldName" : , - "maxStringBytes" : # (optional, defaults to 1024) + "maxStringBytes" : # (optional, defaults to 1024), + "filterNullValues" : # (optional, defaults to false) } ``` @@ -202,7 +203,8 @@ Note that queries with first/last aggregators on a segment created with rollup e "type" : "stringLast", "name" : , "fieldName" : , - "maxStringBytes" : # (optional, defaults to 1024) + "maxStringBytes" : # (optional, defaults to 1024), + "filterNullValues" : # (optional, defaults to false) } ``` From 33c944f261ad93c1279484246c21fe8a25059237 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 10:46:49 +0200 Subject: [PATCH 38/45] Fix checkstyle --- .../query/aggregation/first/LongFirstAggregatorFactory.java | 2 +- .../query/aggregation/first/StringFirstAggregatorFactory.java | 2 +- .../druid/query/aggregation/last/LongLastAggregatorFactory.java | 2 +- .../aggregation/first/StringFirstBufferAggregatorTest.java | 2 -- 4 files changed, 3 insertions(+), 5 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java index a2780e94e6a4..0b2d81ef555b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/LongFirstAggregatorFactory.java @@ -23,8 +23,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.primitives.Longs; -import io.druid.java.util.common.StringUtils; import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.UOE; import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 34244eb3d1f7..081d3db05ce3 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -216,7 +216,7 @@ public boolean equals(Object o) StringFirstAggregatorFactory that = (StringFirstAggregatorFactory) o; return fieldName.equals(that.fieldName) && name.equals(that.name) && - maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; + maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java index 4bee9e9a6659..07d6674d2c1c 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/LongLastAggregatorFactory.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import io.druid.java.util.common.StringUtils; import io.druid.collections.SerializablePair; +import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.UOE; import io.druid.query.aggregation.AggregateCombiner; import io.druid.query.aggregation.Aggregator; diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index 45de88e63bae..9e1bf0621c3e 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -23,8 +23,6 @@ import io.druid.query.aggregation.SerializablePairLongString; import io.druid.query.aggregation.TestLongColumnSelector; import io.druid.query.aggregation.TestObjectColumnSelector; -import io.druid.query.aggregation.last.StringLastAggregatorFactory; -import io.druid.query.aggregation.last.StringLastBufferAggregator; import org.junit.Assert; import org.junit.Test; From acfbed88311bdc55403ae30d33728ca3dd5a0b2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Fri, 27 Jul 2018 10:52:08 +0200 Subject: [PATCH 39/45] Update header license --- .../aggregation/SerializablePairLongString.java | 12 ++++++------ .../aggregation/SerializablePairLongStringSerde.java | 12 ++++++------ .../first/StringFirstAggregateCombiner.java | 12 ++++++------ .../aggregation/first/StringFirstAggregator.java | 12 ++++++------ .../first/StringFirstAggregatorFactory.java | 12 ++++++------ .../first/StringFirstBufferAggregator.java | 12 ++++++------ .../first/StringFirstFoldingAggregatorFactory.java | 12 ++++++------ .../last/StringLastAggregateCombiner.java | 12 ++++++------ .../query/aggregation/last/StringLastAggregator.java | 12 ++++++------ .../last/StringLastAggregatorFactory.java | 12 ++++++------ .../aggregation/last/StringLastBufferAggregator.java | 12 ++++++------ .../last/StringLastFoldingAggregatorFactory.java | 12 ++++++------ .../first/StringFirstAggregationTest.java | 12 ++++++------ .../first/StringFirstBufferAggregatorTest.java | 12 ++++++------ .../first/StringFirstTimeseriesQueryTest.java | 12 ++++++------ .../aggregation/last/StringLastAggregationTest.java | 12 ++++++------ .../last/StringLastBufferAggregatorTest.java | 12 ++++++------ .../last/StringLastTimeseriesQueryTest.java | 12 ++++++------ 18 files changed, 108 insertions(+), 108 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java index 1fba28382178..91f9b2622a75 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongString.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index f63d2d2e70f1..179afe53441e 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java index b694dd0dd9ab..6aaa77a85c14 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index 03421e8d0924..c45e47bae5be 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 081d3db05ce3..8abc87202fd9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index 20b2725692fc..fda04432f071 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index d52725a02777..134f7afd1717 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java index 4e29daa98854..6625f084e792 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregateCombiner.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index 713699d61885..7c68e9ebc141 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index 60090280b8ed..10bc23e0e075 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 94c74db20ca3..405b557aed26 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index f41ff0e3c46f..c0ee9d7751be 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 1be3b3c6cffc..2aa9118862c6 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index 9e1bf0621c3e..d1cf4d4e1dad 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index f6421e662ec0..ab3b34b9dd76 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index 6a31a51a0f63..7cc703e30858 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index d83426903fba..1b3b6161175b 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 45c372ab627b..1ed8b7226106 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ From ff56f3e4a1caf27931511bc9d2d42c3de34f52e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 30 Jul 2018 09:33:05 +0200 Subject: [PATCH 40/45] Fix StringFirstAggregatorFactory.VALUE_COMPARATOR --- .../SerializablePairLongStringSerde.java | 38 +--------------- .../first/StringFirstAggregatorFactory.java | 43 ++++++++++++++----- 2 files changed, 35 insertions(+), 46 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java index 179afe53441e..ca245fa13395 100644 --- a/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/SerializablePairLongStringSerde.java @@ -21,6 +21,7 @@ import io.druid.data.input.InputRow; import io.druid.java.util.common.StringUtils; +import io.druid.query.aggregation.first.StringFirstAggregatorFactory; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -86,42 +87,7 @@ public ObjectStrategy getObjectStrategy() @Override public int compare(@Nullable SerializablePairLongString o1, @Nullable SerializablePairLongString o2) { - int comparation; - - // First we check if the objects are null - if (o1 == null && o2 == null) { - comparation = 0; - } else if (o1 == null) { - comparation = -1; - } else if (o2 == null) { - comparation = 1; - } else { - - // If the objects are not null, we will try to compare using timestamp - comparation = o1.lhs.compareTo(o2.lhs); - - // If both timestamp are the same, we try to compare the Strings - if (comparation == 0) { - - // First we check if the strings are null - if (o1.rhs == null && o2.rhs == null) { - comparation = 0; - } else if (o1.rhs == null) { - comparation = -1; - } else if (o2.rhs == null) { - comparation = 1; - } else { - - // If the strings are not null, we will compare them - // Note: This comparison maybe doesn't make sense to first/last aggregators, - // because compare both SerializablePairLongString based on String maybe doesn't matter - // to define each is first or last. - comparation = o1.rhs.compareTo(o2.rhs); - } - } - } - - return comparation; + return StringFirstAggregatorFactory.VALUE_COMPARATOR.compare(o1, o2); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 8abc87202fd9..22690b1ac9ea 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -50,18 +50,41 @@ public class StringFirstAggregatorFactory extends AggregatorFactory ((SerializablePairLongString) o2).lhs ); - public static final Comparator VALUE_COMPARATOR = (o1, o2) -> { - String s1 = null; - String s2 = null; - - if (o1 != null) { - s1 = ((SerializablePairLongString) o1).rhs; - } - if (o2 != null) { - s2 = ((SerializablePairLongString) o2).rhs; + public static final Comparator VALUE_COMPARATOR = (o1, o2) -> { + int comparation; + + // First we check if the objects are null + if (o1 == null && o2 == null) { + comparation = 0; + } else if (o1 == null) { + comparation = -1; + } else if (o2 == null) { + comparation = 1; + } else { + + // If the objects are not null, we will try to compare using timestamp + comparation = o1.lhs.compareTo(o2.lhs); + + // If both timestamp are the same, we try to compare the Strings + if (comparation == 0) { + + // First we check if the strings are null + if (o1.rhs == null && o2.rhs == null) { + comparation = 0; + } else if (o1.rhs == null) { + comparation = -1; + } else if (o2.rhs == null) { + comparation = 1; + } else { + + // If the strings are not null, we will compare them + // Note: This comparation maybe doesn't make sense to first/last aggregators + comparation = o1.rhs.compareTo(o2.rhs); + } + } } - return Objects.equals(s1, s2) ? 1 : 0; + return comparation; }; private final String fieldName; From 7ff6fc30e01684bbe56afbdefd81f473d79b669e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Mon, 30 Jul 2018 09:38:45 +0200 Subject: [PATCH 41/45] Fix StringFirstAggregatorCombiner --- .../java/io/druid/query/aggregation/AggregateCombiner.java | 2 +- .../query/aggregation/first/StringFirstAggregateCombiner.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java index cfc67c30d96c..e112411b5b11 100644 --- a/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/AggregateCombiner.java @@ -63,7 +63,7 @@ public interface AggregateCombiner extends ColumnValueSelector * become a subject for modification during subsequent fold() calls. * * Since the state of AggregateCombiner is undefined before {@link #reset} is ever called on it, the effects of - * calling fold() are also undefined in this case. The {@link #reset} is called first before {@link #fold} + * calling fold() are also undefined in this case. * * @see AggregatorFactory#combine */ diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java index 6aaa77a85c14..ee0038c8b768 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -37,7 +37,9 @@ public void reset(ColumnValueSelector selector) @Override public void fold(ColumnValueSelector selector) { - // Nothing to do. It needs to keep the first string value. + if (firstString != null) { + firstString = (String) selector.getObject(); + } } @Nullable From b074c5f9a0e885df8a0e4508838aeb029df698b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Tue, 31 Jul 2018 09:58:06 +0200 Subject: [PATCH 42/45] Fix if condition at StringFirstAggregateCombiner --- .../query/aggregation/first/StringFirstAggregateCombiner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java index ee0038c8b768..4c79d617a3bf 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -37,7 +37,7 @@ public void reset(ColumnValueSelector selector) @Override public void fold(ColumnValueSelector selector) { - if (firstString != null) { + if (firstString == null) { firstString = (String) selector.getObject(); } } From 21dfdc1d1b54d820039ac421a680c5725f87499f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Tue, 31 Jul 2018 12:18:44 +0200 Subject: [PATCH 43/45] Remove filterNullValues from string first/last aggregators --- .../first/StringFirstAggregator.java | 7 +- .../first/StringFirstAggregatorFactory.java | 28 ++--- .../first/StringFirstBufferAggregator.java | 7 +- .../StringFirstFoldingAggregatorFactory.java | 11 +- .../last/StringLastAggregator.java | 7 +- .../last/StringLastAggregatorFactory.java | 28 ++--- .../last/StringLastBufferAggregator.java | 7 +- .../StringLastFoldingAggregatorFactory.java | 11 +- .../first/StringFirstAggregationTest.java | 2 +- .../StringFirstBufferAggregatorTest.java | 108 +----------------- .../first/StringFirstTimeseriesQueryTest.java | 4 +- .../last/StringLastAggregationTest.java | 2 +- .../last/StringLastBufferAggregatorTest.java | 108 +----------------- .../last/StringLastTimeseriesQueryTest.java | 4 +- 14 files changed, 50 insertions(+), 284 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java index c45e47bae5be..5710a6107280 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregator.java @@ -31,7 +31,6 @@ public class StringFirstAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; private final int maxStringBytes; - private final boolean filterNullValues; protected long firstTime; protected String firstValue; @@ -39,14 +38,12 @@ public class StringFirstAggregator implements Aggregator public StringFirstAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes, - boolean filterNullValues + int maxStringBytes ) { this.valueSelector = valueSelector; this.timeSelector = timeSelector; this.maxStringBytes = maxStringBytes; - this.filterNullValues = filterNullValues; firstTime = Long.MAX_VALUE; firstValue = null; @@ -75,7 +72,7 @@ public void aggregate() if (firstValue != null && firstValue.length() > maxStringBytes) { firstValue = firstValue.substring(0, maxStringBytes); } - } else if (!filterNullValues) { + } else { firstValue = null; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 22690b1ac9ea..df15af8618c0 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -90,14 +90,12 @@ public class StringFirstAggregatorFactory extends AggregatorFactory private final String fieldName; private final String name; protected final int maxStringBytes; - protected final boolean filterNullValues; @JsonCreator public StringFirstAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes, - @JsonProperty("filterNullValues") Boolean filterNullValues + @JsonProperty("maxStringBytes") Integer maxStringBytes ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); @@ -105,7 +103,6 @@ public StringFirstAggregatorFactory( this.name = name; this.fieldName = fieldName; this.maxStringBytes = maxStringBytes == null ? DEFAULT_MAX_STRING_SIZE : maxStringBytes; - this.filterNullValues = filterNullValues == null ? false : filterNullValues; } @Override @@ -114,8 +111,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) return new StringFirstAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes, - filterNullValues + maxStringBytes ); } @@ -125,8 +121,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) return new StringFirstBufferAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes, - filterNullValues + maxStringBytes ); } @@ -151,13 +146,13 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringFirstFoldingAggregatorFactory(name, name, maxStringBytes, filterNullValues); + return new StringFirstFoldingAggregatorFactory(name, name, maxStringBytes); } @Override public List getRequiredColumns() { - return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes, filterNullValues)); + return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override @@ -192,12 +187,6 @@ public Integer getMaxStringBytes() return maxStringBytes; } - @JsonProperty - public Boolean getFilterNullValues() - { - return filterNullValues; - } - @Override public List requiredFields() { @@ -210,7 +199,6 @@ public byte[] getCacheKey() return new CacheKeyBuilder(AggregatorUtil.STRING_FIRST_CACHE_TYPE_ID) .appendString(fieldName) .appendInt(maxStringBytes) - .appendBoolean(filterNullValues) .build(); } @@ -238,14 +226,13 @@ public boolean equals(Object o) StringFirstAggregatorFactory that = (StringFirstAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name) && - maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; + return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; } @Override public int hashCode() { - return Objects.hash(name, fieldName, maxStringBytes, filterNullValues); + return Objects.hash(name, fieldName, maxStringBytes); } @Override @@ -255,7 +242,6 @@ public String toString() "name='" + name + '\'' + ", fieldName='" + fieldName + '\'' + ", maxStringBytes=" + maxStringBytes + '\'' + - ", filterNullValues=" + filterNullValues + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java index fda04432f071..c71cfbfc2de7 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstBufferAggregator.java @@ -34,19 +34,16 @@ public class StringFirstBufferAggregator implements BufferAggregator private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; private final int maxStringBytes; - private final boolean filterNullValues; public StringFirstBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes, - boolean filterNullValues + int maxStringBytes ) { this.timeSelector = timeSelector; this.valueSelector = valueSelector; this.maxStringBytes = maxStringBytes; - this.filterNullValues = filterNullValues; } @Override @@ -97,7 +94,7 @@ public void aggregate(ByteBuffer buf, int position) mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.put(valueBytes); - } else if (!filterNullValues) { + } else { mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, 0); } diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java index 134f7afd1717..b268bafa5ae9 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstFoldingAggregatorFactory.java @@ -37,18 +37,17 @@ public class StringFirstFoldingAggregatorFactory extends StringFirstAggregatorFa public StringFirstFoldingAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes, - @JsonProperty("filterNullValues") Boolean filterNullValues + @JsonProperty("maxStringBytes") Integer maxStringBytes ) { - super(name, fieldName, maxStringBytes, filterNullValues); + super(name, fieldName, maxStringBytes); } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringFirstAggregator(null, null, maxStringBytes, filterNullValues) + return new StringFirstAggregator(null, null, maxStringBytes) { @Override public void aggregate() @@ -66,7 +65,7 @@ public void aggregate() public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringFirstBufferAggregator(null, null, maxStringBytes, filterNullValues) + return new StringFirstBufferAggregator(null, null, maxStringBytes) { @Override public void aggregate(ByteBuffer buf, int position) @@ -88,7 +87,7 @@ public void aggregate(ByteBuffer buf, int position) mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.put(valueBytes); - } else if (!filterNullValues) { + } else { mutationBuffer.putInt(position + Long.BYTES, 0); } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java index 7c68e9ebc141..85cd0dddb3e6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregator.java @@ -31,7 +31,6 @@ public class StringLastAggregator implements Aggregator private final BaseObjectColumnValueSelector valueSelector; private final BaseLongColumnValueSelector timeSelector; private final int maxStringBytes; - private final boolean filterNullValues; protected long lastTime; protected String lastValue; @@ -39,14 +38,12 @@ public class StringLastAggregator implements Aggregator public StringLastAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes, - boolean filterNullValues + int maxStringBytes ) { this.valueSelector = valueSelector; this.timeSelector = timeSelector; this.maxStringBytes = maxStringBytes; - this.filterNullValues = filterNullValues; lastTime = Long.MIN_VALUE; lastValue = null; @@ -75,7 +72,7 @@ public void aggregate() if (lastValue != null && lastValue.length() > maxStringBytes) { lastValue = lastValue.substring(0, maxStringBytes); } - } else if (!filterNullValues) { + } else { lastValue = null; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index 10bc23e0e075..9fb0773d46dc 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -47,14 +47,12 @@ public class StringLastAggregatorFactory extends AggregatorFactory private final String fieldName; private final String name; protected final int maxStringBytes; - protected final boolean filterNullValues; @JsonCreator public StringLastAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes, - @JsonProperty("filterNullValues") Boolean filterNullValues + @JsonProperty("maxStringBytes") Integer maxStringBytes ) { Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name"); @@ -64,7 +62,6 @@ public StringLastAggregatorFactory( this.maxStringBytes = maxStringBytes == null ? StringFirstAggregatorFactory.DEFAULT_MAX_STRING_SIZE : maxStringBytes; - this.filterNullValues = filterNullValues == null ? false : filterNullValues; } @Override @@ -73,8 +70,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) return new StringLastAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes, - filterNullValues + maxStringBytes ); } @@ -84,8 +80,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) return new StringLastBufferAggregator( metricFactory.makeColumnValueSelector(Column.TIME_COLUMN_NAME), metricFactory.makeColumnValueSelector(fieldName), - maxStringBytes, - filterNullValues + maxStringBytes ); } @@ -110,13 +105,13 @@ public AggregateCombiner makeAggregateCombiner() @Override public AggregatorFactory getCombiningFactory() { - return new StringLastFoldingAggregatorFactory(name, name, maxStringBytes, filterNullValues); + return new StringLastFoldingAggregatorFactory(name, name, maxStringBytes); } @Override public List getRequiredColumns() { - return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes, filterNullValues)); + return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override @@ -151,12 +146,6 @@ public Integer getMaxStringBytes() return maxStringBytes; } - @JsonProperty - public Boolean getFilterNullValues() - { - return filterNullValues; - } - @Override public List requiredFields() { @@ -169,7 +158,6 @@ public byte[] getCacheKey() return new CacheKeyBuilder(AggregatorUtil.STRING_LAST_CACHE_TYPE_ID) .appendString(fieldName) .appendInt(maxStringBytes) - .appendBoolean(filterNullValues) .build(); } @@ -197,14 +185,13 @@ public boolean equals(Object o) StringLastAggregatorFactory that = (StringLastAggregatorFactory) o; - return fieldName.equals(that.fieldName) && name.equals(that.name) && - maxStringBytes == that.maxStringBytes && filterNullValues == that.filterNullValues; + return fieldName.equals(that.fieldName) && name.equals(that.name) && maxStringBytes == that.maxStringBytes; } @Override public int hashCode() { - return Objects.hash(name, fieldName, maxStringBytes, filterNullValues); + return Objects.hash(name, fieldName, maxStringBytes); } @Override @@ -214,7 +201,6 @@ public String toString() "name='" + name + '\'' + ", fieldName='" + fieldName + '\'' + ", maxStringBytes=" + maxStringBytes + '\'' + - ", filterNullValues=" + filterNullValues + '\'' + '}'; } } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java index 405b557aed26..12c99483a61b 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastBufferAggregator.java @@ -34,19 +34,16 @@ public class StringLastBufferAggregator implements BufferAggregator private final BaseLongColumnValueSelector timeSelector; private final BaseObjectColumnValueSelector valueSelector; private final int maxStringBytes; - private final boolean filterNullValues; public StringLastBufferAggregator( BaseLongColumnValueSelector timeSelector, BaseObjectColumnValueSelector valueSelector, - int maxStringBytes, - boolean filterNullValues + int maxStringBytes ) { this.timeSelector = timeSelector; this.valueSelector = valueSelector; this.maxStringBytes = maxStringBytes; - this.filterNullValues = filterNullValues; } @Override @@ -97,7 +94,7 @@ public void aggregate(ByteBuffer buf, int position) mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.put(valueBytes); - } else if (!filterNullValues) { + } else { mutationBuffer.putLong(position, time); mutationBuffer.putInt(position + Long.BYTES, 0); } diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java index c0ee9d7751be..9bd6a64488ee 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastFoldingAggregatorFactory.java @@ -37,18 +37,17 @@ public class StringLastFoldingAggregatorFactory extends StringLastAggregatorFact public StringLastFoldingAggregatorFactory( @JsonProperty("name") String name, @JsonProperty("fieldName") final String fieldName, - @JsonProperty("maxStringBytes") Integer maxStringBytes, - @JsonProperty("filterNullValues") Boolean filterNullValues + @JsonProperty("maxStringBytes") Integer maxStringBytes ) { - super(name, fieldName, maxStringBytes, filterNullValues); + super(name, fieldName, maxStringBytes); } @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringLastAggregator(null, null, maxStringBytes, filterNullValues) + return new StringLastAggregator(null, null, maxStringBytes) { @Override public void aggregate() @@ -66,7 +65,7 @@ public void aggregate() public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { final BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(getName()); - return new StringLastBufferAggregator(null, null, maxStringBytes, filterNullValues) + return new StringLastBufferAggregator(null, null, maxStringBytes) { @Override public void aggregate(ByteBuffer buf, int position) @@ -86,7 +85,7 @@ public void aggregate(ByteBuffer buf, int position) mutationBuffer.putInt(position + Long.BYTES, valueBytes.length); mutationBuffer.position(position + Long.BYTES + Integer.BYTES); mutationBuffer.put(valueBytes); - } else if (!filterNullValues) { + } else { mutationBuffer.putInt(position + Long.BYTES, 0); } } diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java index 2aa9118862c6..8f523c02ee59 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -55,7 +55,7 @@ public class StringFirstAggregationTest @Before public void setup() { - stringLastAggFactory = new StringFirstAggregatorFactory("billy", "nilly", MAX_STRING_SIZE, false); + stringLastAggFactory = new StringFirstAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); combiningAggFactory = (StringFirstAggregatorFactory) stringLastAggFactory.getCombiningFactory(); timeSelector = new TestLongColumnSelector(times); valueSelector = new TestObjectColumnSelector<>(strings); diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java index d1cf4d4e1dad..8a4a0de986eb 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstBufferAggregatorTest.java @@ -51,20 +51,18 @@ public void testBufferAggregate() throws Exception final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -97,20 +95,18 @@ public void testNullBufferAggregate() throws Exception final long[] timestamps = {2222L, 1111L, 3333L, 4444L, 5555L}; final String[] strings = {null, "AAAA", "BBBB", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -143,20 +139,18 @@ public void testNoStringValue() final long[] timestamps = {1526724000L, 1526724600L}; final Double[] doubles = {null, 2.00}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringFirstBufferAggregator agg = new StringFirstBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -174,94 +168,4 @@ public void testNoStringValue() aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); } } - - @Test - public void testFilterNullValues() throws Exception - { - - final long[] timestamps = {1111L, 2222L, 3333L, 1110L, 5555L}; - final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; - Integer maxStringBytes = 1024; - Boolean filterNullValues = true; - - TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); - TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); - - StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues - ); - - StringFirstBufferAggregator agg = new StringFirstBufferAggregator( - longColumnSelector, - objectColumnSelector, - maxStringBytes, - filterNullValues - ); - - String testString = "ZZZZ"; - - ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); - buf.putLong(1526728500L); - buf.putInt(testString.length()); - buf.put(testString.getBytes(StandardCharsets.UTF_8)); - - int position = 0; - - agg.init(buf, position); - //noinspection ForLoopReplaceableByForEach - for (int i = 0; i < timestamps.length; i++) { - aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); - } - - SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - - - Assert.assertEquals("expectec last string value", strings[0], sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[0]), new Long(sp.lhs)); - } - - @Test - public void testNotFilterNullValues() throws Exception - { - - final long[] timestamps = {1111L, 2222L, 3333L, 1110L, 5555L}; - final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; - Integer maxStringBytes = 1024; - Boolean filterNullValues = false; - - TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); - TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); - - StringFirstAggregatorFactory factory = new StringFirstAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues - ); - - StringFirstBufferAggregator agg = new StringFirstBufferAggregator( - longColumnSelector, - objectColumnSelector, - maxStringBytes, - filterNullValues - ); - - String testString = "ZZZZ"; - - ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); - buf.putLong(1526728500L); - buf.putInt(testString.length()); - buf.put(testString.getBytes(StandardCharsets.UTF_8)); - - int position = 0; - - agg.init(buf, position); - //noinspection ForLoopReplaceableByForEach - for (int i = 0; i < timestamps.length; i++) { - aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); - } - - SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - - - Assert.assertEquals("expectec last string value", strings[3], sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[3]), new Long(sp.lhs)); - } } diff --git a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index ab3b34b9dd76..bac9a6dfcbc3 100644 --- a/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -59,7 +59,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .withQueryGranularity(Granularities.SECOND) .withMetrics(new CountAggregatorFactory("cnt")) .withMetrics(new StringFirstAggregatorFactory( - "last_client_type", "client_type", 1024, false) + "last_client_type", "client_type", 1024) ) .build() ) @@ -101,7 +101,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .aggregators( Lists.newArrayList( new StringFirstAggregatorFactory( - "last_client_type", client_type, 1024, false + "last_client_type", client_type, 1024 ) ) ) diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java index 7cc703e30858..1f2ecc48152f 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastAggregationTest.java @@ -55,7 +55,7 @@ public class StringLastAggregationTest @Before public void setup() { - stringLastAggFactory = new StringLastAggregatorFactory("billy", "nilly", MAX_STRING_SIZE, false); + stringLastAggFactory = new StringLastAggregatorFactory("billy", "nilly", MAX_STRING_SIZE); combiningAggFactory = (StringLastAggregatorFactory) stringLastAggFactory.getCombiningFactory(); timeSelector = new TestLongColumnSelector(times); valueSelector = new TestObjectColumnSelector<>(strings); diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java index 1b3b6161175b..c7c125b67ddf 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastBufferAggregatorTest.java @@ -51,20 +51,18 @@ public void testBufferAggregate() throws Exception final long[] timestamps = {1526724600L, 1526724700L, 1526724800L, 1526725900L, 1526725000L}; final String[] strings = {"AAAA", "BBBB", "CCCC", "DDDD", "EEEE"}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -97,20 +95,18 @@ public void testNullBufferAggregate() throws Exception final long[] timestamps = {1111L, 2222L, 6666L, 4444L, 5555L}; final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -143,20 +139,18 @@ public void testNoStringValue() final long[] timestamps = {1526724000L, 1526724600L}; final Double[] doubles = {null, 2.00}; Integer maxStringBytes = 1024; - Boolean filterNullValues = false; TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(doubles); StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues + "billy", "billy", maxStringBytes ); StringLastBufferAggregator agg = new StringLastBufferAggregator( longColumnSelector, objectColumnSelector, - maxStringBytes, - filterNullValues + maxStringBytes ); String testString = "ZZZZ"; @@ -174,94 +168,4 @@ public void testNoStringValue() aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); } } - - @Test - public void testFilterNullValues() throws Exception - { - - final long[] timestamps = {1111L, 2222L, 3333L, 6666L, 5555L}; - final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; - Integer maxStringBytes = 1024; - Boolean filterNullValues = true; - - TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); - TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); - - StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues - ); - - StringLastBufferAggregator agg = new StringLastBufferAggregator( - longColumnSelector, - objectColumnSelector, - maxStringBytes, - filterNullValues - ); - - String testString = "ZZZZ"; - - ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); - buf.putLong(1526728500L); - buf.putInt(testString.length()); - buf.put(testString.getBytes(StandardCharsets.UTF_8)); - - int position = 0; - - agg.init(buf, position); - //noinspection ForLoopReplaceableByForEach - for (int i = 0; i < timestamps.length; i++) { - aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); - } - - SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - - - Assert.assertEquals("expectec last string value", strings[4], sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[4]), new Long(sp.lhs)); - } - - @Test - public void testNotFilterNullValues() throws Exception - { - - final long[] timestamps = {1111L, 2222L, 3333L, 6666L, 5555L}; - final String[] strings = {"CCCC", "AAAA", "BBBB", null, "EEEE"}; - Integer maxStringBytes = 1024; - Boolean filterNullValues = false; - - TestLongColumnSelector longColumnSelector = new TestLongColumnSelector(timestamps); - TestObjectColumnSelector objectColumnSelector = new TestObjectColumnSelector<>(strings); - - StringLastAggregatorFactory factory = new StringLastAggregatorFactory( - "billy", "billy", maxStringBytes, filterNullValues - ); - - StringLastBufferAggregator agg = new StringLastBufferAggregator( - longColumnSelector, - objectColumnSelector, - maxStringBytes, - filterNullValues - ); - - String testString = "ZZZZ"; - - ByteBuffer buf = ByteBuffer.allocate(factory.getMaxIntermediateSize()); - buf.putLong(1526728500L); - buf.putInt(testString.length()); - buf.put(testString.getBytes(StandardCharsets.UTF_8)); - - int position = 0; - - agg.init(buf, position); - //noinspection ForLoopReplaceableByForEach - for (int i = 0; i < timestamps.length; i++) { - aggregateBuffer(longColumnSelector, objectColumnSelector, agg, buf, position); - } - - SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); - - - Assert.assertEquals("expectec last string value", strings[3], sp.rhs); - Assert.assertEquals("last string timestamp is the biggest", new Long(timestamps[3]), new Long(sp.lhs)); - } } diff --git a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 1ed8b7226106..a68798e64be0 100644 --- a/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -59,7 +59,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .withQueryGranularity(Granularities.SECOND) .withMetrics(new CountAggregatorFactory("cnt")) .withMetrics(new StringLastAggregatorFactory( - "last_client_type", "client_type", 1024, false) + "last_client_type", "client_type", 1024) ) .build() ) @@ -101,7 +101,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .aggregators( Lists.newArrayList( new StringLastAggregatorFactory( - "last_client_type", client_type, 1024, false + "last_client_type", client_type, 1024 ) ) ) From 19b55c378d6f5600e30abefd34a3433750a628e5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Tue, 31 Jul 2018 23:18:34 +0200 Subject: [PATCH 44/45] Add isReset flag in FirstAggregatorCombiner --- .../aggregation/first/StringFirstAggregateCombiner.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java index 4c79d617a3bf..20487f659002 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregateCombiner.java @@ -27,18 +27,21 @@ public class StringFirstAggregateCombiner extends ObjectAggregateCombiner { private String firstString; + private boolean isReset = false; @Override public void reset(ColumnValueSelector selector) { firstString = (String) selector.getObject(); + isReset = true; } @Override public void fold(ColumnValueSelector selector) { - if (firstString == null) { + if (!isReset) { firstString = (String) selector.getObject(); + isReset = true; } } From f6742121b3eecab2b0f665fae6e96216a53fbec4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9s=20Gomez?= Date: Tue, 31 Jul 2018 23:18:54 +0200 Subject: [PATCH 45/45] Change Arrays.asList to Collections.singletonList --- .../query/aggregation/first/StringFirstAggregatorFactory.java | 3 ++- .../query/aggregation/last/StringLastAggregatorFactory.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java index df15af8618c0..187e89154812 100644 --- a/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -35,6 +35,7 @@ import io.druid.segment.column.Column; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -152,7 +153,7 @@ public AggregatorFactory getCombiningFactory() @Override public List getRequiredColumns() { - return Arrays.asList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Collections.singletonList(new StringFirstAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override diff --git a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java index 9fb0773d46dc..cb4f36366122 100644 --- a/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/io/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -35,6 +35,7 @@ import io.druid.segment.column.Column; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -111,7 +112,7 @@ public AggregatorFactory getCombiningFactory() @Override public List getRequiredColumns() { - return Arrays.asList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); + return Collections.singletonList(new StringLastAggregatorFactory(fieldName, fieldName, maxStringBytes)); } @Override