diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimal.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimal.java index 6168d3fc03a..440f20e0d64 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimal.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimal.java @@ -20,6 +20,8 @@ package org.apache.druid.compressedbigdecimal; +import com.google.common.base.Preconditions; + import java.math.BigDecimal; import java.math.BigInteger; @@ -28,11 +30,14 @@ import java.math.BigInteger; */ public class ArrayCompressedBigDecimal extends CompressedBigDecimal { + public static final CompressedBigDecimal ZERO_COMPRESSED_BIG_DECIMAL = + new ArrayCompressedBigDecimal(BigDecimal.ZERO); private static final int BYTE_MASK = 0xff; private final int[] array; + /** * Construct an AccumulatingBigDecimal using the referenced initial * value and scale. @@ -73,9 +78,9 @@ public class ArrayCompressedBigDecimal extends CompressedBigDecimal for (int ii = 0; ii < arrayLen; ++ii) { this.array[ii] = (BYTE_MASK & bytes[--bytesIdx]) | - (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 8 | - (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 16 | - (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 24; + (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 8 | + (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 16 | + (bytesIdx != 0 ? (BYTE_MASK & bytes[--bytesIdx]) : (((int) bytes[0]) >> 8)) << 24; } } @@ -121,18 +126,53 @@ public class ArrayCompressedBigDecimal extends CompressedBigDecimal } /** - * Allocate a new CompressedBigDecimal with the specified size and scale. + * Allocate a new CompressedBigDecimal with the specified size and scale and a value of 0 * * @param size size of the int array used for calculations * @param scale scale of the number * @return CompressedBigDecimal */ - public static ArrayCompressedBigDecimal allocate(int size, int scale) + public static ArrayCompressedBigDecimal allocateZero(int size, int scale) { int[] arr = new int[size]; + return new ArrayCompressedBigDecimal(arr, scale); } + /** + * Allocate a new CompressedBigDecimal with the specified size and scale and a value of "MIN_VALUE" + * + * @param size size of the int array used for calculations + * @param scale scale of the number + * @return CompressedBigDecimal + */ + public static ArrayCompressedBigDecimal allocateMin(int size, int scale) + { + int[] arr = new int[size]; + ArrayCompressedBigDecimal result = new ArrayCompressedBigDecimal(arr, scale); + + result.setMinValue(); + + return result; + } + + /** + * Allocate a new CompressedBigDecimal with the specified size and scale and a value of "MAX_VALUE" + * + * @param size size of the int array used for calculations + * @param scale scale of the number + * @return CompressedBigDecimal + */ + public static ArrayCompressedBigDecimal allocateMax(int size, int scale) + { + int[] arr = new int[size]; + ArrayCompressedBigDecimal result = new ArrayCompressedBigDecimal(arr, scale); + + result.setMaxValue(); + + return result; + } + @Override public CompressedBigDecimal toHeap() { @@ -182,4 +222,29 @@ public class ArrayCompressedBigDecimal extends CompressedBigDecimal array[idx] = val; } + @Override + protected void setValue(CompressedBigDecimal rhs) + { + Preconditions.checkArgument( + rhs.getArraySize() <= array.length, + "lhs too small to store entry: lhs [%s] vs rhs [%s]", + getArraySize(), + rhs.getArraySize() + ); + + long extension = rhs.getArrayEntry(rhs.getArraySize() - 1) < 0 ? INT_MASK : 0L; + + for (int i = 0; i < array.length; i++) { + long rhsElement; + + if (i < rhs.getArraySize()) { + rhsElement = INT_MASK & rhs.getArrayEntry(i); + } else { + rhsElement = extension; + } + + array[i] = (int) rhsElement; + } + + } } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimal.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimal.java index b0727a8f04f..e2d6e6b7815 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimal.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimal.java @@ -19,6 +19,8 @@ package org.apache.druid.compressedbigdecimal; +import com.google.common.base.Preconditions; + import java.nio.ByteBuffer; /** @@ -26,7 +28,6 @@ import java.nio.ByteBuffer; */ public class ByteBufferCompressedBigDecimal extends CompressedBigDecimal { - private final ByteBuffer buf; private final int position; private final int size; @@ -66,6 +67,36 @@ public class ByteBufferCompressedBigDecimal extends CompressedBigDecimal copyToBuffer(buf, position, size, val); } + public static void initZero(ByteBuffer buf, int position, int size) + { + for (int i = 0; i < size; ++i) { + buf.putInt(position + (i * Integer.BYTES), 0); + } + } + + public static void initMin(ByteBuffer buf, int position, int size) + { + for (int i = 0; i < size; ++i) { + if (i == size - 1) { + buf.putInt(position + (i * Integer.BYTES), 0x80000000); + } else { + buf.putInt(position + (i * Integer.BYTES), 0); + } + } + } + + public static void initMax(ByteBuffer buf, int position, int size) + { + for (int i = 0; i < size; ++i) { + if (i == size - 1) { + buf.putInt(position + (i * Integer.BYTES), 0x7FFFFFFF); + } else { + buf.putInt(position + (i * Integer.BYTES), 0xFFFFFFFF); + } + } + } + + @Override public CompressedBigDecimal toHeap() { @@ -105,6 +136,30 @@ public class ByteBufferCompressedBigDecimal extends CompressedBigDecimal buf.putInt(position + idx * Integer.BYTES, val); } + @Override + protected void setValue(CompressedBigDecimal rhs) + { + Preconditions.checkArgument( + rhs.getArraySize() <= getArraySize(), + "lhs too small to store entry: lhs [%s] vs rhs [%s]", + size, + rhs.getArraySize() + ); + + long extension = rhs.getArrayEntry(rhs.getArraySize() - 1) < 0 ? INT_MASK : 0L; + + for (int i = 0; i < size; i++) { + long rhsElement; + + if (i < rhs.getArraySize()) { + rhsElement = INT_MASK & rhs.getArrayEntry(i); + } else { + rhsElement = extension; + } + + buf.putInt(position + i * Integer.BYTES, (int) rhsElement); + } + } /** * Copy a compressed big decimal into a Bytebuffer in a format understood by this class. @@ -123,5 +178,4 @@ public class ByteBufferCompressedBigDecimal extends CompressedBigDecimal buf.putInt(position + ii * Integer.BYTES, val.getArrayEntry(ii)); } } - } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimal.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimal.java index 926ad3b19bb..3b3fdb7719e 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimal.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimal.java @@ -19,6 +19,7 @@ package org.apache.druid.compressedbigdecimal; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; import java.math.BigDecimal; @@ -32,8 +33,7 @@ import java.util.function.ToIntBiFunction; */ public abstract class CompressedBigDecimal extends Number implements Comparable { - - private static final long INT_MASK = 0x00000000ffffffffL; + protected static final long INT_MASK = 0x00000000ffffffffL; private final int scale; @@ -60,11 +60,10 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< * @param rhs The object to accumulate * @return a reference to this */ - public CompressedBigDecimal accumulate(CompressedBigDecimal rhs) + public CompressedBigDecimal accumulateSum(CompressedBigDecimal rhs) { - if (rhs.scale != scale) { - throw new IllegalArgumentException("Cannot accumulate MutableBigDecimals with differing scales"); - } + checkScaleCompatibility(rhs); + if (rhs.getArraySize() > getArraySize()) { throw new IllegalArgumentException("Right hand side too big to fit in the result value"); } @@ -74,17 +73,57 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< return this; } + public CompressedBigDecimal accumulateMax(CompressedBigDecimal rhs) + { + checkScaleCompatibility(rhs); + + if (compareTo(rhs) < 0) { + setValue(rhs); + } + + return this; + } + + public CompressedBigDecimal accumulateMin(CompressedBigDecimal rhs) + { + checkScaleCompatibility(rhs); + + if (compareTo(rhs) > 0) { + setValue(rhs); + } + + return this; + } + + private void checkScaleCompatibility(CompressedBigDecimal rhs) + { + Preconditions.checkArgument( + rhs.getScale() == getScale(), + "scales do not match: lhs [%s] vs rhs [%s]", + getScale(), + rhs.getScale() + ); + } + /** - * Clear any accumulated value, resetting to zero. Scale is preserved at its original value. + * copy the value from the rhs into this object + *

+ * Note: implementations in subclasses are virtually identical, but specialized to allow for inlining of + * element access. Callsites to the rhs's getArrayEntry should be monomorphic also (each subclass should only see the + * same type it is) * - * @return this + * @param rhs a {@link CompressedBigDecimal} object */ - public CompressedBigDecimal reset() + protected abstract void setValue(CompressedBigDecimal rhs); + + /** + * Clear any value, resetting to zero. Scale is preserved at its original value. + */ + public void reset() { for (int ii = 0; ii < getArraySize(); ++ii) { setArrayEntry(ii, 0); } - return this; } /** @@ -141,14 +180,20 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< * * @return the byte array for use in BigInteger */ - private byte[] toByteArray() + private ByteArrayResult toByteArray() { int byteArrayLength = getArraySize() * 4; byte[] bytes = new byte[byteArrayLength]; int byteIdx = 0; + boolean isZero = true; for (int ii = getArraySize(); ii > 0; --ii) { int val = getArrayEntry(ii - 1); + + if (val != 0) { + isZero = false; + } + bytes[byteIdx + 3] = (byte) val; val >>>= 8; bytes[byteIdx + 2] = (byte) val; @@ -168,10 +213,32 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< // one is on a byte boundary). emptyBytes--; } - return Arrays.copyOfRange(bytes, emptyBytes, byteArrayLength); + return new ByteArrayResult(Arrays.copyOfRange(bytes, emptyBytes, byteArrayLength), isZero); } - return bytes; + return new ByteArrayResult(bytes, isZero); + } + + protected void setMinValue() + { + for (int i = 0; i < getArraySize(); i++) { + if (i == getArraySize() - 1) { + setArrayEntry(i, 0x80000000); + } else { + setArrayEntry(i, 0); + } + } + } + + protected void setMaxValue() + { + for (int i = 0; i < getArraySize(); i++) { + if (i == getArraySize() - 1) { + setArrayEntry(i, 0x7FFFFFFF); + } else { + setArrayEntry(i, 0xFFFFFFFF); + } + } } /** @@ -220,17 +287,22 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< */ public BigDecimal toBigDecimal() { - BigInteger bigInt = new BigInteger(toByteArray()); - return new BigDecimal(bigInt, scale); + ByteArrayResult byteArrayResult = toByteArray(); + + if (byteArrayResult.isZero) { + return new BigDecimal(BigDecimal.ZERO.toBigInteger(), 0); + } else { + BigInteger bigInt = new BigInteger(byteArrayResult.bytes); + + return new BigDecimal(bigInt, scale); + } } - /* (non-Javadoc) - * @see java.lang.Object#toString() - */ @Override public String toString() { - return toBigDecimal().toString(); + BigDecimal bigDecimal = toBigDecimal(); + return bigDecimal.toString(); } /** @@ -248,9 +320,23 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< return getArrayEntry(getArraySize() - 1) < 0; } - public boolean isPositive() + public boolean isNonNegative() { - return !isNegative(); + return getArrayEntry(getArraySize() - 1) >= 0; + } + + public boolean isZero() + { + boolean isZero = true; + + for (int i = getArraySize() - 1; i >= 0; i--) { + if (getArrayEntry(i) != 0) { + isZero = false; + break; + } + } + + return isZero; } /** @@ -315,9 +401,9 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< { // this short-circuit serves two functions: 1. it speeds up comparison in +/- cases 2. it avoids the case of // overflow of positive - negative and negative - positive. p - p and n - n both fit in the given allotment of ints - if (lhs.isPositive() && rhs.isNegative()) { + if (lhs.isNonNegative() && rhs.isNegative()) { return 1; - } else if (lhs.isNegative() && rhs.isPositive()) { + } else if (lhs.isNegative() && rhs.isNonNegative()) { return -1; } @@ -421,4 +507,16 @@ public abstract class CompressedBigDecimal extends Number implements Comparable< { return toBigDecimal().doubleValue(); } + + private static class ByteArrayResult + { + private final byte[] bytes; + private final boolean isZero; + + public ByteArrayResult(byte[] bytes, boolean isZero) + { + this.bytes = bytes; + this.isZero = isZero; + } + } } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombinerBase.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombinerBase.java new file mode 100644 index 00000000000..377d150b733 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombinerBase.java @@ -0,0 +1,75 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + + +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +public abstract class CompressedBigDecimalAggregateCombinerBase implements AggregateCombiner +{ + protected CompressedBigDecimal value; + + private final String className; + + protected CompressedBigDecimalAggregateCombinerBase(String className) + { + this.className = className; + } + + @Override + public abstract void reset(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector); + + @Override + public abstract void fold(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector); + + @Override + public double getDouble() + { + throw new UnsupportedOperationException(className + " does not support getDouble()"); + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException(className + " does not support getFloat()"); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException(className + " does not support getLong()"); + } + + @Nullable + @Override + public CompressedBigDecimal getObject() + { + return value; + } + + @Override + public Class classOfObject() + { + return CompressedBigDecimal.class; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorBase.java similarity index 55% rename from extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregator.java rename to extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorBase.java index 2d4d81ad574..03388278152 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregator.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorBase.java @@ -25,12 +25,14 @@ import org.apache.druid.segment.ColumnValueSelector; /** * An Aggregator to aggregate big decimal values. */ -public class CompressedBigDecimalAggregator implements Aggregator +public abstract class CompressedBigDecimalAggregatorBase implements Aggregator { + protected final ColumnValueSelector selector; + protected final boolean strictNumberParsing; + protected final CompressedBigDecimal value; + protected boolean empty; - private final ColumnValueSelector selector; - private final boolean strictNumberParsing; - private final CompressedBigDecimal sum; + private final String className; /** * Constructor. @@ -40,64 +42,44 @@ public class CompressedBigDecimalAggregator implements Aggregator * @param selector that has the metric value * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 */ - public CompressedBigDecimalAggregator( + protected CompressedBigDecimalAggregatorBase( int size, int scale, ColumnValueSelector selector, - boolean strictNumberParsing + boolean strictNumberParsing, + String className ) { this.selector = selector; this.strictNumberParsing = strictNumberParsing; - this.sum = ArrayCompressedBigDecimal.allocate(size, scale); + this.className = className; + value = initValue(size, scale); + empty = true; } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.Aggregator#aggregate() - */ + protected abstract CompressedBigDecimal initValue(int size, int scale); + @Override - public void aggregate() - { - CompressedBigDecimal selectedObject = Utils.objToCompressedBigDecimal(selector.getObject(), strictNumberParsing); + public abstract void aggregate(); - if (selectedObject != null) { - if (selectedObject.getScale() != sum.getScale()) { - selectedObject = Utils.scale(selectedObject, sum.getScale()); - } - sum.accumulate(selectedObject); - } - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.Aggregator#get() - */ @Override public Object get() { - return sum; + return empty ? null : value; } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.Aggregator#getFloat() - */ @Override public float getFloat() { - throw new UnsupportedOperationException("CompressedBigDecimalAggregator does not support getFloat()"); + throw new UnsupportedOperationException(className + " does not support getFloat()"); } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.Aggregator#getLong() - */ @Override public long getLong() { - throw new UnsupportedOperationException("CompressedBigDecimalAggregator does not support getLong()"); + throw new UnsupportedOperationException(className + " does not support getLong()"); } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.Aggregator#close() - */ @Override public void close() { diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactory.java deleted file mode 100644 index 88df1fc782a..00000000000 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactory.java +++ /dev/null @@ -1,337 +0,0 @@ -/* - * 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. 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 - * - * 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 org.apache.druid.compressedbigdecimal; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Objects; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.aggregation.AggregateCombiner; -import org.apache.druid.query.aggregation.Aggregator; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.BufferAggregator; -import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory; -import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; - -import javax.annotation.Nonnull; -import javax.annotation.Nullable; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - - -/** - * An aggregator factory to generate longSum aggregator object. - */ -public class CompressedBigDecimalAggregatorFactory - extends NullableNumericAggregatorFactory> -{ - - public static final int DEFAULT_SCALE = 9; - public static final int DEFAULT_SIZE = 6; - public static final boolean DEFAULT_STRICT_NUMBER_PARSING = false; - - private static final byte CACHE_TYPE_ID = 0x37; - - public static final Comparator COMPARATOR = CompressedBigDecimal::compareTo; - - private final String name; - private final String fieldName; - private final int size; - private final int scale; - private final boolean strictNumberParsing; - - /** - * Constructor. - * - * @param name metric field name - * @param fieldName fieldName metric field name - * @param size size of the int array used for calculations - * @param scale scale of the number - * @param strictNumberParsing if true, failure to parse strings to numbers throws an exception. otherwise 0 is - * returned - */ - @JsonCreator - public CompressedBigDecimalAggregatorFactory( - @JsonProperty("name") String name, - @JsonProperty("fieldName") String fieldName, - @JsonProperty(value = "size", required = false) Integer size, - @JsonProperty(value = "scale", required = false) Integer scale, - @JsonProperty(value = "strictNumberParsing", required = false) Boolean strictNumberParsing - ) - { - this.name = name; - this.fieldName = fieldName; - this.size = size == null ? DEFAULT_SIZE : size; - this.scale = scale == null ? DEFAULT_SCALE : scale; - this.strictNumberParsing = strictNumberParsing == null ? DEFAULT_STRICT_NUMBER_PARSING : strictNumberParsing; - } - - @SuppressWarnings("unchecked") - @Override - protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) - { - return (ColumnValueSelector) metricFactory.makeColumnValueSelector(fieldName); - } - - @Override - protected Aggregator factorize( - ColumnSelectorFactory metricFactory, - @Nonnull ColumnValueSelector selector - ) - { - return new CompressedBigDecimalAggregator(size, scale, selector, strictNumberParsing); - } - - @Override - protected BufferAggregator factorizeBuffered( - ColumnSelectorFactory metricFactory, - @Nonnull ColumnValueSelector selector - ) - { - return new CompressedBigDecimalBufferAggregator(size, scale, selector, strictNumberParsing); - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getComparator() - */ - @Override - public Comparator getComparator() - { - return COMPARATOR; - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#combine(java.lang.Object, java.lang.Object) - */ - @Nullable - @Override - public Object combine(Object lhs, Object rhs) - { - if (lhs == null && rhs == null) { - return ArrayCompressedBigDecimal.allocate(size, scale); - } else if (lhs == null) { - return rhs; - } else if (rhs == null) { - return lhs; - } else { - // Allocate a new result and accumlate both left and right into it. - // This ensures that the result has the correct scale, avoiding possible IllegalArgumentExceptions - // due to truncation when the deserialized objects aren't big enough to hold the accumlated result. - // The most common case this avoids is deserializing 0E-9 into a CompressedBigDecimal with array - // size 1 and then accumulating a larger value into it. - CompressedBigDecimal retVal = ArrayCompressedBigDecimal.allocate(size, scale); - CompressedBigDecimal left = (CompressedBigDecimal) lhs; - CompressedBigDecimal right = (CompressedBigDecimal) rhs; - if (left.signum() != 0) { - retVal.accumulate(left); - } - if (right.signum() != 0) { - retVal.accumulate(right); - } - return retVal; - } - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getCombiningFactory() - */ - @Override - public AggregatorFactory getCombiningFactory() - { - return new CompressedBigDecimalAggregatorFactory(name, name, size, scale, strictNumberParsing); - } - - @Override - public AggregateCombiner makeAggregateCombiner() - { - return new CompressedBigDecimalAggregateCombiner(); - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getRequiredColumns() - */ - @Override - public List getRequiredColumns() - { - return Collections.singletonList(new CompressedBigDecimalAggregatorFactory( - fieldName, - fieldName, - size, - scale, - strictNumberParsing - )); - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#deserialize(java.lang.Object) - */ - @Nullable - @Override - public Object deserialize(Object object) - { - if (object == null) { - return null; - } else if (object instanceof BigDecimal) { - return new ArrayCompressedBigDecimal((BigDecimal) object); - } else if (object instanceof Double) { - return new ArrayCompressedBigDecimal(new BigDecimal((Double) object)); - } else if (object instanceof String) { - return new ArrayCompressedBigDecimal(new BigDecimal((String) object)); - } else { - throw new RuntimeException("unknown type in deserialize: " + object.getClass().getSimpleName()); - } - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#requiredFields() - */ - @Override - public List requiredFields() - { - return Collections.singletonList(fieldName); - } - - /* (non-Javadoc) Get Type */ - @Override - public ValueType getType() - { - return ValueType.COMPLEX; - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getTypeName() - */ - @Override - public String getComplexTypeName() - { - return CompressedBigDecimalModule.COMPRESSED_BIG_DECIMAL; - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getCacheKey() - */ - @Override - public byte[] getCacheKey() - { - byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); - return ByteBuffer.allocate(1 + fieldNameBytes.length).put(CACHE_TYPE_ID).put(fieldNameBytes).array(); - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#finalizeComputation(java.lang.Object) - */ - @Override - public Object finalizeComputation(Object object) - { - CompressedBigDecimal compressedBigDecimal = (CompressedBigDecimal) object; - BigDecimal bigDecimal = compressedBigDecimal.toBigDecimal(); - return bigDecimal.compareTo(BigDecimal.ZERO) == 0 ? 0 : bigDecimal; - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getName() - */ - @Override - @JsonProperty - public String getName() - { - return name; - } - - /** - * Get the filed name. - * - * @return dimension/metric field name - */ - @JsonProperty - public String getFieldName() - { - return fieldName; - } - - @JsonProperty - public int getScale() - { - return scale; - } - - @JsonProperty - public int getSize() - { - return size; - } - - @JsonProperty - public boolean getStrictNumberParsing() - { - return strictNumberParsing; - } - - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.AggregatorFactory#getMaxIntermediateSize() - */ - @Override - public int getMaxIntermediateSize() - { - return Integer.BYTES * size; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - CompressedBigDecimalAggregatorFactory that = (CompressedBigDecimalAggregatorFactory) o; - return size == that.size - && scale == that.scale - && Objects.equal(name, that.name) - && Objects.equal(fieldName, that.fieldName) - && Objects.equal(strictNumberParsing, that.strictNumberParsing); - } - - @Override - public int hashCode() - { - return Objects.hashCode(name, fieldName, size, scale, strictNumberParsing); - } - - @Override - public String toString() - { - return "CompressedBigDecimalSumAggregatorFactory{" + - "name='" + getName() + '\'' + - ", type='" + getComplexTypeName() + '\'' + - ", fieldName='" + getFieldName() + '\'' + - ", requiredFields='" + requiredFields() + '\'' + - ", size='" + getSize() + '\'' + - ", scale='" + getScale() + '\'' + - ", strictNumberParsing='" + getStrictNumberParsing() + '\'' + - '}'; - } -} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java new file mode 100644 index 00000000000..201ceee405e --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryBase.java @@ -0,0 +1,232 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; + + +public abstract class CompressedBigDecimalAggregatorFactoryBase + extends NullableNumericAggregatorFactory> +{ + public static final int DEFAULT_SCALE = 9; + public static final int DEFAULT_SIZE = 6; + public static final boolean DEFAULT_STRICT_NUMBER_PARSING = false; + public static final int BUFFER_AGGREGATOR_HEADER_SIZE_BYTES = 1; + + + public static final Comparator COMPARATOR = CompressedBigDecimal::compareTo; + + protected final String name; + protected final String fieldName; + protected final int size; + protected final int scale; + protected final boolean strictNumberParsing; + + private final byte[] cacheKey; + + /** + * Constructor. + * + * @param name metric field name + * @param fieldName fieldName metric field name + * @param size size of the int array used for calculations + * @param scale scale of the number + * @param strictNumberParsing if true, failure to parse strings to numbers throws an exception. otherwise 0 is + * returned + * @param cacheTypeId + */ + protected CompressedBigDecimalAggregatorFactoryBase( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size, + @JsonProperty(value = "scale", required = false) Integer scale, + @JsonProperty(value = "strictNumberParsing", required = false) Boolean strictNumberParsing, + byte cacheTypeId + ) + { + this.name = name; + this.fieldName = fieldName; + this.size = size == null ? DEFAULT_SIZE : size; + this.scale = scale == null ? DEFAULT_SCALE : scale; + this.strictNumberParsing = strictNumberParsing == null ? DEFAULT_STRICT_NUMBER_PARSING : strictNumberParsing; + + byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); + + cacheKey = ByteBuffer.allocate(1 + fieldNameBytes.length + 2 * Integer.BYTES + 1) + .put(cacheTypeId) + .put(fieldNameBytes) + .putInt(this.size) + .putInt(this.scale) + .put((byte) (this.strictNumberParsing ? 1 : 0)) + .array(); + } + + @SuppressWarnings("unchecked") + @Override + public ColumnValueSelector selector(ColumnSelectorFactory metricFactory) + { + return (ColumnValueSelector) metricFactory.makeColumnValueSelector(fieldName); + } + + @Override + protected abstract Aggregator factorize( + ColumnSelectorFactory metricFactory, + ColumnValueSelector selector + ); + @Override + protected abstract BufferAggregator factorizeBuffered( + ColumnSelectorFactory metricFactory, + ColumnValueSelector selector + ); + + @Override + public Comparator getComparator() + { + return COMPARATOR; + } + + @Nullable + @Override + public abstract Object combine(Object lhs, Object rhs); + + @Override + public abstract AggregatorFactory getCombiningFactory(); + + @Override + public abstract AggregateCombiner makeAggregateCombiner(); + + @Override + public abstract List getRequiredColumns(); + + @Override + public abstract String toString(); + + @Nullable + @Override + public Object deserialize(Object object) + { + return Utils.objToCompressedBigDecimal(object); + } + + @Override + public List requiredFields() + { + return Collections.singletonList(fieldName); + } + + @Override + public ColumnType getIntermediateType() + { + return ColumnType.ofComplex(CompressedBigDecimalModule.COMPRESSED_BIG_DECIMAL); + } + + @Override + public byte[] getCacheKey() + { + return cacheKey; + } + + @Override + public Object finalizeComputation(Object object) + { + return object; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + /** + * Get the filed name. + * + * @return dimension/metric field name + */ + @JsonProperty + public String getFieldName() + { + return fieldName; + } + + @JsonProperty + public int getScale() + { + return scale; + } + + @JsonProperty + public int getSize() + { + return size; + } + + @JsonProperty + public boolean getStrictNumberParsing() + { + return strictNumberParsing; + } + + @Override + public int getMaxIntermediateSize() + { + return BUFFER_AGGREGATOR_HEADER_SIZE_BYTES + Integer.BYTES * size; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompressedBigDecimalAggregatorFactoryBase that = (CompressedBigDecimalAggregatorFactoryBase) o; + return size == that.size + && scale == that.scale + && Objects.equal(name, that.name) + && Objects.equal(fieldName, that.fieldName) + && Objects.equal(strictNumberParsing, that.strictNumberParsing); + } + + @Override + public int hashCode() + { + return Objects.hashCode(name, fieldName, size, scale, strictNumberParsing); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryCreator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryCreator.java new file mode 100644 index 00000000000..0d7269251b9 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorFactoryCreator.java @@ -0,0 +1,33 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.aggregation.AggregatorFactory; + +public interface CompressedBigDecimalAggregatorFactoryCreator +{ + AggregatorFactory create( + String name, + String fieldName, + Integer size, + Integer scale, + Boolean strictNumberParsing + ); +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregatorBase.java similarity index 54% rename from extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregator.java rename to extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregatorBase.java index 494973f42b3..f06fc781d81 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregator.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalBufferAggregatorBase.java @@ -24,18 +24,14 @@ import org.apache.druid.segment.ColumnValueSelector; import java.nio.ByteBuffer; -/** - * A buffered aggregator to aggregate big decimal value. - */ -public class CompressedBigDecimalBufferAggregator implements BufferAggregator +public abstract class CompressedBigDecimalBufferAggregatorBase implements BufferAggregator { + protected final ColumnValueSelector selector; + protected final int size; + protected final int scale; + protected final boolean strictNumberParsing; - //Cache will hold the aggregated value. - //We are using ByteBuffer to hold the key to the aggregated value. - private final ColumnValueSelector selector; - private final int size; - private final int scale; - private boolean strictNumberParsing; + private final int headerSizeBytes; /** * Constructor. @@ -44,52 +40,63 @@ public class CompressedBigDecimalBufferAggregator implements BufferAggregator * @param scale the scale * @param selector a ColumnSelector to retrieve incoming values * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 - * */ - public CompressedBigDecimalBufferAggregator( + * @param headerSizeBytes size of a header, if it exists + */ + public CompressedBigDecimalBufferAggregatorBase( int size, int scale, ColumnValueSelector selector, - boolean strictNumberParsing + boolean strictNumberParsing, + int headerSizeBytes ) { this.selector = selector; this.size = size; this.scale = scale; this.strictNumberParsing = strictNumberParsing; + this.headerSizeBytes = headerSizeBytes; } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#init(java.nio.ByteBuffer, int) - */ @Override public void init(ByteBuffer buf, int position) { - for (int ii = 0; ii < size; ++ii) { - buf.putInt(position + (ii * Integer.BYTES), 0); - } + ByteBufferCompressedBigDecimal.initMin(buf, position + headerSizeBytes, size); + setEmpty(true, buf, position); } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#aggregate(java.nio.ByteBuffer, int) - */ @Override public void aggregate(ByteBuffer buf, int position) { - CompressedBigDecimal addend = Utils.objToCompressedBigDecimal(selector.getObject(), strictNumberParsing); + CompressedBigDecimal addend = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + scale, + strictNumberParsing + ); + if (addend != null) { - Utils.accumulate(buf, position, size, scale, addend); + setEmpty(false, buf, position); + + CompressedBigDecimal existing = new ByteBufferCompressedBigDecimal( + buf, + position + headerSizeBytes, + size, + scale + ); + + existing.accumulateMax(addend); } } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#get(java.nio.ByteBuffer, int) - */ @Override public Object get(ByteBuffer buf, int position) { + if (isEmpty(buf, position)) { + return null; + } + ByteBufferCompressedBigDecimal byteBufferCompressedBigDecimal = new ByteBufferCompressedBigDecimal( buf, - position, + position + headerSizeBytes, size, scale ); @@ -99,30 +106,41 @@ public class CompressedBigDecimalBufferAggregator implements BufferAggregator return heapCompressedBigDecimal; } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#getFloat(java.nio.ByteBuffer, int) - */ @Override public float getFloat(ByteBuffer buf, int position) { - throw new UnsupportedOperationException("CompressedBigDecimalBufferAggregator does not support getFloat()"); + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not support getFloat()"); } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#getLong(java.nio.ByteBuffer, int) - */ @Override public long getLong(ByteBuffer buf, int position) { - throw new UnsupportedOperationException("CompressedBigDecimalBufferAggregator does not support getLong()"); + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not support getLong()"); } - /* (non-Javadoc) - * @see org.apache.druid.query.aggregation.BufferAggregator#close() - */ @Override public void close() { // no resources to cleanup } + + //do not allow overriding to avoid megamorphic callsites in this class and subclasses + protected final void setEmpty(boolean value, ByteBuffer byteBuffer, int position) + { + // no header means the aggregator is considered non-empty + if (headerSizeBytes > 0) { + byteBuffer.put(position, (byte) (value ? 1 : 0)); + } + } + + //do not allow overriding to avoid megamorphic callsites in this class and subclasses + protected final boolean isEmpty(ByteBuffer byteBuffer, int position) + { + // no header means the aggregator is considered non-empty + if (headerSizeBytes > 0) { + return byteBuffer.get(position) != 0; + } else { + return false; + } + } } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalJsonSerializer.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalJsonSerializer.java index 4d9cb92aa8e..1a3263785bf 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalJsonSerializer.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalJsonSerializer.java @@ -31,10 +31,6 @@ import java.io.IOException; public class CompressedBigDecimalJsonSerializer extends JsonSerializer { - /* (non-Javadoc) - * @see JsonSerializer#serialize(java.lang.Object, com.fasterxml.jackson.core.JsonGenerator, - * com.fasterxml.jackson.databind.SerializerProvider) - */ @Override public void serialize(CompressedBigDecimal value, JsonGenerator jgen, SerializerProvider provider) throws IOException diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregateCombiner.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregateCombiner.java new file mode 100644 index 00000000000..073485f80d3 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregateCombiner.java @@ -0,0 +1,64 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * AggregateCombiner for CompressedBigDecimals computing a max + */ +public class CompressedBigDecimalMaxAggregateCombiner extends CompressedBigDecimalAggregateCombinerBase +{ + public CompressedBigDecimalMaxAggregateCombiner() + { + super(CompressedBigDecimalMaxAggregateCombiner.class.getSimpleName()); + } + + @Override + public void reset(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector) + { + @SuppressWarnings("unchecked") + ColumnValueSelector selector = + (ColumnValueSelector) columnValueSelector; + + CompressedBigDecimal cbd = selector.getObject(); + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); + } else { + value.setValue(cbd); + } + } + + @Override + public void fold(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector) + { + @SuppressWarnings("unchecked") + ColumnValueSelector selector = + (ColumnValueSelector) columnValueSelector; + CompressedBigDecimal cbd = selector.getObject(); + + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); + } else { + value.accumulateMax(cbd); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregator.java new file mode 100644 index 00000000000..de6538f9d5b --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregator.java @@ -0,0 +1,68 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * An Aggregator to aggregate big decimal values. + */ +public class CompressedBigDecimalMaxAggregator extends CompressedBigDecimalAggregatorBase +{ + + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector that has the metric value + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalMaxAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, CompressedBigDecimalMaxAggregator.class.getSimpleName()); + } + + @Override + protected CompressedBigDecimal initValue(int size, int scale) + { + return ArrayCompressedBigDecimal.allocateMin(size, scale); + } + + @Override + public void aggregate() + { + CompressedBigDecimal selectedObject = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + value.getScale(), + strictNumberParsing + ); + + if (selectedObject != null) { + empty = false; + value.accumulateMax(selectedObject); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java new file mode 100644 index 00000000000..47fbda0ec63 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorFactory.java @@ -0,0 +1,141 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; + +import java.util.Collections; +import java.util.List; + +public class CompressedBigDecimalMaxAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase +{ + private static final byte CACHE_TYPE_ID = 0x37; + + /** + * Constructor. + * + * @param name metric field name + * @param fieldName fieldName metric field name + * @param size size of the int array used for calculations + * @param scale scale of the number + * @param strictNumberParsing if true, failure to parse strings to numbers throws an exception. otherwise 0 is + * returned + */ + @JsonCreator + public CompressedBigDecimalMaxAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size, + @JsonProperty(value = "scale", required = false) Integer scale, + @JsonProperty(value = "strictNumberParsing", required = false) Boolean strictNumberParsing + ) + { + super(name, fieldName, size, scale, strictNumberParsing, CACHE_TYPE_ID); + } + + @Override + protected Aggregator factorize( + ColumnSelectorFactory metricFactory, + ColumnValueSelector selector + ) + { + return new CompressedBigDecimalMaxAggregator(size, scale, selector, strictNumberParsing); + } + + @Override + protected BufferAggregator factorizeBuffered( + ColumnSelectorFactory metricFactory, + ColumnValueSelector selector + ) + { + return new CompressedBigDecimalMaxBufferAggregator(size, scale, selector, strictNumberParsing); + } + + @Override + public Object combine(Object lhs, Object rhs) + { + if (lhs == null && rhs == null) { + return null; + } else if (lhs == null) { + return rhs; + } else if (rhs == null) { + return lhs; + } else { + // Allocate a new result and accumlate both left and right into it. + // This ensures that the result has the correct scale, avoiding possible IllegalArgumentExceptions + // due to truncation when the deserialized objects aren't big enough to hold the accumlated result. + // The most common case this avoids is deserializing 0E-9 into a CompressedBigDecimal with array + // size 1 and then accumulating a larger value into it. + CompressedBigDecimal retVal = ArrayCompressedBigDecimal.allocateMin(size, scale); + CompressedBigDecimal left = (CompressedBigDecimal) lhs; + CompressedBigDecimal right = (CompressedBigDecimal) rhs; + + retVal.accumulateMax(left); + retVal.accumulateMax(right); + + return retVal; + } + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new CompressedBigDecimalMaxAggregatorFactory(name, name, size, scale, strictNumberParsing); + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new CompressedBigDecimalMaxAggregateCombiner(); + } + + @Override + public List getRequiredColumns() + { + return Collections.singletonList(new CompressedBigDecimalMaxAggregatorFactory( + fieldName, + fieldName, + size, + scale, + strictNumberParsing + )); + } + + @Override + public String toString() + { + return "CompressedBigDecimalMaxAggregatorFactory{" + + "name='" + getName() + '\'' + + ", type='" + getIntermediateType().asTypeString() + '\'' + + ", fieldName='" + getFieldName() + '\'' + + ", requiredFields='" + requiredFields() + '\'' + + ", size='" + getSize() + '\'' + + ", scale='" + getScale() + '\'' + + ", strictNumberParsing='" + getStrictNumberParsing() + '\'' + + '}'; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxBufferAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxBufferAggregator.java new file mode 100644 index 00000000000..59fd281d30b --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxBufferAggregator.java @@ -0,0 +1,80 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; + +/** + * A buffered aggregator to compute a max CompressedBigDecimal + */ +public class CompressedBigDecimalMaxBufferAggregator extends CompressedBigDecimalBufferAggregatorBase +{ + private static final int HEADER_SIZE_BYTES = + CompressedBigDecimalAggregatorFactoryBase.BUFFER_AGGREGATOR_HEADER_SIZE_BYTES; + + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector a ColumnSelector to retrieve incoming values + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalMaxBufferAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, HEADER_SIZE_BYTES); + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBufferCompressedBigDecimal.initMin(buf, position + HEADER_SIZE_BYTES, size); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + CompressedBigDecimal addend = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + scale, + strictNumberParsing + ); + + if (addend != null) { + setEmpty(false, buf, position); + + CompressedBigDecimal existing = new ByteBufferCompressedBigDecimal( + buf, + position + HEADER_SIZE_BYTES, + size, + scale + ); + + existing.accumulateMax(addend); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregator.java new file mode 100644 index 00000000000..8a941cafdd1 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregator.java @@ -0,0 +1,30 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalMaxSqlAggregator extends CompressedBigDecimalSqlAggregatorBase +{ + static final String NAME = "BIG_MAX"; + + public CompressedBigDecimalMaxSqlAggregator() + { + super(NAME, CompressedBigDecimalMaxAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregateCombiner.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregateCombiner.java new file mode 100644 index 00000000000..f38e5b80531 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregateCombiner.java @@ -0,0 +1,64 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * AggregateCombiner for CompressedBigDecimals computing a min + */ +public class CompressedBigDecimalMinAggregateCombiner extends CompressedBigDecimalAggregateCombinerBase +{ + public CompressedBigDecimalMinAggregateCombiner() + { + super(CompressedBigDecimalMinAggregator.class.getSimpleName()); + } + + @Override + public void reset(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector) + { + @SuppressWarnings("unchecked") + ColumnValueSelector selector = + (ColumnValueSelector) columnValueSelector; + + CompressedBigDecimal cbd = selector.getObject(); + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); + } else { + value.setValue(cbd); + } + } + + @Override + public void fold(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector) + { + @SuppressWarnings("unchecked") + ColumnValueSelector selector = + (ColumnValueSelector) columnValueSelector; + CompressedBigDecimal cbd = selector.getObject(); + + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); + } else { + value.accumulateMin(cbd); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregator.java new file mode 100644 index 00000000000..11aa3c76974 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregator.java @@ -0,0 +1,67 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * An Aggregator to compute min bigDecimal values + */ +public class CompressedBigDecimalMinAggregator extends CompressedBigDecimalAggregatorBase +{ + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector that has the metric value + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalMinAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, CompressedBigDecimalMinAggregator.class.getSimpleName()); + } + + @Override + protected CompressedBigDecimal initValue(int size, int scale) + { + return ArrayCompressedBigDecimal.allocateMax(size, scale); + } + + @Override + public void aggregate() + { + CompressedBigDecimal selectedObject = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + value.getScale(), + strictNumberParsing + ); + + if (selectedObject != null) { + empty = false; + value.accumulateMin(selectedObject); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java new file mode 100644 index 00000000000..8d3590a7329 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorFactory.java @@ -0,0 +1,146 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nonnull; +import java.util.Collections; +import java.util.List; + + +/** + * An aggregator factory to generate min of BigDecimal aggregator object. + */ +public class CompressedBigDecimalMinAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase +{ + private static final byte CACHE_TYPE_ID = 0x38; + + /** + * Constructor. + * + * @param name metric field name + * @param fieldName fieldName metric field name + * @param size size of the int array used for calculations + * @param scale scale of the number + * @param strictNumberParsing if true, failure to parse strings to numbers throws an exception. otherwise 0 is + * returned + */ + @JsonCreator + public CompressedBigDecimalMinAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size, + @JsonProperty(value = "scale", required = false) Integer scale, + @JsonProperty(value = "strictNumberParsing", required = false) Boolean strictNumberParsing + ) + { + super(name, fieldName, size, scale, strictNumberParsing, CACHE_TYPE_ID); + } + + @Override + public Aggregator factorize( + ColumnSelectorFactory metricFactory, + @Nonnull ColumnValueSelector selector + ) + { + return new CompressedBigDecimalMinAggregator(size, scale, selector, strictNumberParsing); + } + + @Override + public BufferAggregator factorizeBuffered( + ColumnSelectorFactory metricFactory, + @Nonnull ColumnValueSelector selector + ) + { + return new CompressedBigDecimalMinBufferAggregator(size, scale, selector, strictNumberParsing); + } + + @Override + public Object combine(Object lhs, Object rhs) + { + if (lhs == null && rhs == null) { + return null; + } else if (lhs == null) { + return rhs; + } else if (rhs == null) { + return lhs; + } else { + // Allocate a new result and accumlate both left and right into it. + // This ensures that the result has the correct scale, avoiding possible IllegalArgumentExceptions + // due to truncation when the deserialized objects aren't big enough to hold the accumlated result. + // The most common case this avoids is deserializing 0E-9 into a CompressedBigDecimal with array + // size 1 and then accumulating a larger value into it. + CompressedBigDecimal retVal = ArrayCompressedBigDecimal.allocateMax(size, scale); + CompressedBigDecimal left = (CompressedBigDecimal) lhs; + CompressedBigDecimal right = (CompressedBigDecimal) rhs; + + retVal.accumulateMin(left); + retVal.accumulateMin(right); + + return retVal; + } + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new CompressedBigDecimalMinAggregatorFactory(name, name, size, scale, strictNumberParsing); + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new CompressedBigDecimalMinAggregateCombiner(); + } + + @Override + public List getRequiredColumns() + { + return Collections.singletonList(new CompressedBigDecimalMinAggregatorFactory( + name, + fieldName, + size, + scale, + strictNumberParsing + )); + } + + @Override + public String toString() + { + return "CompressedBigDecimalMinAggregatorFactory{" + + "name='" + getName() + '\'' + + ", type='" + getIntermediateType().asTypeString() + '\'' + + ", fieldName='" + getFieldName() + '\'' + + ", requiredFields='" + requiredFields() + '\'' + + ", size='" + getSize() + '\'' + + ", scale='" + getScale() + '\'' + + ", strictNumberParsing='" + getStrictNumberParsing() + '\'' + + '}'; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinBufferAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinBufferAggregator.java new file mode 100644 index 00000000000..3b9e7fb7c48 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinBufferAggregator.java @@ -0,0 +1,81 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; + +/** + * A buffered aggregator to compute a min CompressedBigDecimal + */ +public class CompressedBigDecimalMinBufferAggregator extends CompressedBigDecimalBufferAggregatorBase +{ + private static final int HEADER_OFFSET_BYTES = + CompressedBigDecimalAggregatorFactoryBase.BUFFER_AGGREGATOR_HEADER_SIZE_BYTES; + + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector a ColumnSelector to retrieve incoming values + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalMinBufferAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, HEADER_OFFSET_BYTES); + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBufferCompressedBigDecimal.initMax(buf, position + HEADER_OFFSET_BYTES, size); + setEmpty(true, buf, position); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + CompressedBigDecimal addend = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + scale, + strictNumberParsing + ); + + if (addend != null) { + setEmpty(false, buf, position); + + CompressedBigDecimal existing = new ByteBufferCompressedBigDecimal( + buf, + position + HEADER_OFFSET_BYTES, + size, + scale + ); + + existing.accumulateMin(addend); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregator.java new file mode 100644 index 00000000000..520d306c3a3 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregator.java @@ -0,0 +1,30 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalMinSqlAggregator extends CompressedBigDecimalSqlAggregatorBase +{ + static final String NAME = "BIG_MIN"; + + public CompressedBigDecimalMinSqlAggregator() + { + super(NAME, CompressedBigDecimalMinAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalModule.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalModule.java index 2a885bf8d2a..5f4c844426e 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalModule.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalModule.java @@ -36,12 +36,17 @@ import java.util.List; public class CompressedBigDecimalModule implements DruidModule { public static final String COMPRESSED_BIG_DECIMAL = "compressedBigDecimal"; + public static final String COMPRESSED_BIG_DECIMAL_SUM = "compressedBigDecimalSum"; + public static final String COMPRESSED_BIG_DECIMAL_MAX = "compressedBigDecimalMax"; + public static final String COMPRESSED_BIG_DECIMAL_MIN = "compressedBigDecimalMin"; @Override public void configure(Binder binder) { registerSerde(); - SqlBindings.addAggregator(binder, CompressedBigDecimalSqlAggregator.class); + SqlBindings.addAggregator(binder, CompressedBigDecimalSumSqlAggregator.class); + SqlBindings.addAggregator(binder, CompressedBigDecimalMaxSqlAggregator.class); + SqlBindings.addAggregator(binder, CompressedBigDecimalMinSqlAggregator.class); } public static void registerSerde() @@ -56,13 +61,10 @@ public class CompressedBigDecimalModule implements DruidModule { return ImmutableList.of( new SimpleModule("CompressedBigDecimalModule") - .registerSubtypes( - new NamedType(CompressedBigDecimalAggregatorFactory.class, COMPRESSED_BIG_DECIMAL) - ) - .addSerializer( - CompressedBigDecimal.class, - new CompressedBigDecimalJsonSerializer() - ) + .registerSubtypes(new NamedType(CompressedBigDecimalSumAggregatorFactory.class, COMPRESSED_BIG_DECIMAL_SUM)) + .registerSubtypes(new NamedType(CompressedBigDecimalMaxAggregatorFactory.class, COMPRESSED_BIG_DECIMAL_MAX)) + .registerSubtypes(new NamedType(CompressedBigDecimalMinAggregatorFactory.class, COMPRESSED_BIG_DECIMAL_MIN)) + .addSerializer(CompressedBigDecimal.class, new CompressedBigDecimalJsonSerializer()) ); } } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java similarity index 85% rename from extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregator.java rename to extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java index 643b65343d6..e88bdcae0ab 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregator.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorBase.java @@ -47,15 +47,24 @@ import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import javax.annotation.Nullable; import java.util.List; -public class CompressedBigDecimalSqlAggregator implements SqlAggregator +public abstract class CompressedBigDecimalSqlAggregatorBase implements SqlAggregator { - private static final SqlAggFunction FUNCTION_INSTANCE = new CompressedBigDecimalSqlAggFunction(); - private static final String NAME = "BIG_SUM"; + private final SqlAggFunction sqlAggFunction; + private final CompressedBigDecimalAggregatorFactoryCreator factoryCreator; + + protected CompressedBigDecimalSqlAggregatorBase( + String name, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) + { + this.sqlAggFunction = new CompressedBigDecimalSqlAggFunction(name); + this.factoryCreator = factoryCreator; + } @Override public SqlAggFunction calciteFunction() { - return FUNCTION_INSTANCE; + return sqlAggFunction; } @Nullable @@ -139,7 +148,7 @@ public class CompressedBigDecimalSqlAggregator implements SqlAggregator } // create the factory - AggregatorFactory aggregatorFactory = new CompressedBigDecimalAggregatorFactory( + AggregatorFactory aggregatorFactory = factoryCreator.create( StringUtils.format("%s:agg", name), sumColumnName, size, @@ -152,28 +161,28 @@ public class CompressedBigDecimalSqlAggregator implements SqlAggregator private static class CompressedBigDecimalSqlAggFunction extends SqlAggFunction { - private static final String SIGNATURE2 = "'" + NAME + "'(column, size)"; - private static final String SIGNATURE3 = "'" + NAME + "'(column, size, scale)"; - private static final String SIGNATURE4 = "'" + NAME + "'(column, size, scale, strictNumberParsing)"; - - CompressedBigDecimalSqlAggFunction() + private CompressedBigDecimalSqlAggFunction(String name) { super( - NAME, + name, null, SqlKind.OTHER_FUNCTION, - ReturnTypes.explicit(SqlTypeName.OTHER), + ReturnTypes.explicit(SqlTypeName.VARCHAR), null, OperandTypes.or( // first signature is the colum only, BIG_SUM(column) OperandTypes.and(OperandTypes.ANY, OperandTypes.family(SqlTypeFamily.ANY)), OperandTypes.and( - OperandTypes.sequence(SIGNATURE2, OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL), + OperandTypes.sequence( + "'" + name + "'(column, size)", + OperandTypes.ANY, + OperandTypes.POSITIVE_INTEGER_LITERAL + ), OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.EXACT_NUMERIC) ), OperandTypes.and( OperandTypes.sequence( - SIGNATURE3, + "'" + name + "'(column, size, scale)", OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL, OperandTypes.POSITIVE_INTEGER_LITERAL @@ -182,7 +191,7 @@ public class CompressedBigDecimalSqlAggregator implements SqlAggregator ), OperandTypes.and( OperandTypes.sequence( - SIGNATURE4, + "'" + name + "'(column, size, scale, strictNumberParsing)", OperandTypes.ANY, OperandTypes.POSITIVE_INTEGER_LITERAL, OperandTypes.POSITIVE_INTEGER_LITERAL, diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombiner.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregateCombiner.java similarity index 60% rename from extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombiner.java rename to extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregateCombiner.java index 110c73cb42a..03bf84348c9 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregateCombiner.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregateCombiner.java @@ -20,17 +20,17 @@ package org.apache.druid.compressedbigdecimal; -import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.segment.ColumnValueSelector; -import javax.annotation.Nullable; - /** * AggregateCombiner for CompressedBigDecimals. */ -public class CompressedBigDecimalAggregateCombiner implements AggregateCombiner +public class CompressedBigDecimalSumAggregateCombiner extends CompressedBigDecimalAggregateCombinerBase { - private CompressedBigDecimal sum; + public CompressedBigDecimalSumAggregateCombiner() + { + super(CompressedBigDecimalSumAggregateCombiner.class.getSimpleName()); + } @Override public void reset(@SuppressWarnings("rawtypes") ColumnValueSelector columnValueSelector) @@ -40,11 +40,10 @@ public class CompressedBigDecimalAggregateCombiner implements AggregateCombiner< (ColumnValueSelector) columnValueSelector; CompressedBigDecimal cbd = selector.getObject(); - if (sum == null) { - sum = new ArrayCompressedBigDecimal(cbd); + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); } else { - sum.reset(); - sum.accumulate(cbd); + value.setValue(cbd); } } @@ -57,43 +56,12 @@ public class CompressedBigDecimalAggregateCombiner implements AggregateCombiner< CompressedBigDecimal cbd = selector.getObject(); - if (sum == null) { - sum = new ArrayCompressedBigDecimal(cbd); + if (value == null) { + value = new ArrayCompressedBigDecimal(cbd); } else { if (cbd.signum() != 0) { - sum.accumulate(cbd); + value.accumulateSum(cbd); } } } - - @Override - public double getDouble() - { - throw new UnsupportedOperationException("CompressedBigDecimalCombiner does not support getDouble()"); - } - - @Override - public float getFloat() - { - throw new UnsupportedOperationException("CompressedBigDecimalCombiner does not support getFloat()"); - } - - @Override - public long getLong() - { - throw new UnsupportedOperationException("CompressedBigDecimalCombiner does not support getLong()"); - } - - @Nullable - @Override - public CompressedBigDecimal getObject() - { - return sum; - } - - @Override - public Class classOfObject() - { - return CompressedBigDecimal.class; - } } diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregator.java new file mode 100644 index 00000000000..53f7f9fcd1f --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregator.java @@ -0,0 +1,67 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * An Aggregator to aggregate big decimal values. + */ +public class CompressedBigDecimalSumAggregator extends CompressedBigDecimalAggregatorBase +{ + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector that has the metric value + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalSumAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, CompressedBigDecimalSumAggregator.class.getSimpleName()); + empty = false; + } + + @Override + protected CompressedBigDecimal initValue(int size, int scale) + { + return ArrayCompressedBigDecimal.allocateZero(size, scale); + } + + @Override + public void aggregate() + { + CompressedBigDecimal selectedObject = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + value.getScale(), + strictNumberParsing + ); + + if (selectedObject != null) { + value.accumulateSum(selectedObject); + } + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java new file mode 100644 index 00000000000..bbe5d360813 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorFactory.java @@ -0,0 +1,146 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; + +public class CompressedBigDecimalSumAggregatorFactory extends CompressedBigDecimalAggregatorFactoryBase +{ + private static final byte CACHE_TYPE_ID = 0x39; + + /** + * Constructor. + * + * @param name metric field name + * @param fieldName fieldName metric field name + * @param size size of the int array used for calculations + * @param scale scale of the number + * @param strictNumberParsing if true, failure to parse strings to numbers throws an exception. otherwise 0 is + * returned + */ + @JsonCreator + public CompressedBigDecimalSumAggregatorFactory( + @JsonProperty("name") String name, + @JsonProperty("fieldName") String fieldName, + @JsonProperty(value = "size", required = false) Integer size, + @JsonProperty(value = "scale", required = false) Integer scale, + @JsonProperty(value = "strictNumberParsing", required = false) Boolean strictNumberParsing + ) + { + super(name, fieldName, size, scale, strictNumberParsing, CACHE_TYPE_ID); + } + + @Override + protected Aggregator factorize( + ColumnSelectorFactory metricFactory, + @Nonnull ColumnValueSelector selector + ) + { + return new CompressedBigDecimalSumAggregator(size, scale, selector, strictNumberParsing); + } + + @Override + protected BufferAggregator factorizeBuffered( + ColumnSelectorFactory metricFactory, + @Nonnull ColumnValueSelector selector + ) + { + return new CompressedBigDecimalSumBufferAggregator(size, scale, selector, strictNumberParsing); + } + + @Nullable + @Override + public Object combine(Object lhs, Object rhs) + { + if (lhs == null && rhs == null) { + return ArrayCompressedBigDecimal.allocateZero(size, scale); + } else if (lhs == null) { + return rhs; + } else if (rhs == null) { + return lhs; + } else { + // Allocate a new result and accumlate both left and right into it. + // This ensures that the result has the correct scale, avoiding possible IllegalArgumentExceptions + // due to truncation when the deserialized objects aren't big enough to hold the accumlated result. + // The most common case this avoids is deserializing 0E-9 into a CompressedBigDecimal with array + // size 1 and then accumulating a larger value into it. + CompressedBigDecimal retVal = ArrayCompressedBigDecimal.allocateZero(size, scale); + CompressedBigDecimal left = (CompressedBigDecimal) lhs; + CompressedBigDecimal right = (CompressedBigDecimal) rhs; + if (!left.isZero()) { + retVal.accumulateSum(left); + } + if (!right.isZero()) { + retVal.accumulateSum(right); + } + return retVal; + } + } + + @Override + public AggregatorFactory getCombiningFactory() + { + return new CompressedBigDecimalSumAggregatorFactory(name, name, size, scale, strictNumberParsing); + } + + @Override + public AggregateCombiner makeAggregateCombiner() + { + return new CompressedBigDecimalSumAggregateCombiner(); + } + + @Override + public List getRequiredColumns() + { + return Collections.singletonList(new CompressedBigDecimalSumAggregatorFactory( + name, + fieldName, + size, + scale, + strictNumberParsing + )); + } + + @Override + public String toString() + { + return "CompressedBigDecimalSumAggregatorFactory{" + + "name='" + getName() + '\'' + + ", type='" + getIntermediateType().asTypeString() + '\'' + + ", fieldName='" + getFieldName() + '\'' + + ", requiredFields='" + requiredFields() + '\'' + + ", size='" + getSize() + '\'' + + ", scale='" + getScale() + '\'' + + ", strictNumberParsing='" + getStrictNumberParsing() + '\'' + + '}'; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumBufferAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumBufferAggregator.java new file mode 100644 index 00000000000..7b2e3557274 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumBufferAggregator.java @@ -0,0 +1,88 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; + +import java.nio.ByteBuffer; + +/** + * A buffered aggregator to aggregate big decimal value. + */ +public class CompressedBigDecimalSumBufferAggregator extends CompressedBigDecimalBufferAggregatorBase +{ + /** + * Constructor. + * + * @param size the size to allocate + * @param scale the scale + * @param selector a ColumnSelector to retrieve incoming values + * @param strictNumberParsing true => NumberFormatExceptions thrown; false => NumberFormatException returns 0 + */ + public CompressedBigDecimalSumBufferAggregator( + int size, + int scale, + ColumnValueSelector selector, + boolean strictNumberParsing + ) + { + super(size, scale, selector, strictNumberParsing, 0); + } + + @Override + public void init(ByteBuffer buf, int position) + { + ByteBufferCompressedBigDecimal.initZero(buf, position, size); + } + + @Override + public void aggregate(ByteBuffer buf, int position) + { + CompressedBigDecimal addend = Utils.objToCompressedBigDecimalWithScale( + selector.getObject(), + scale, + strictNumberParsing + ); + + if (addend != null) { + CompressedBigDecimal existing = new ByteBufferCompressedBigDecimal(buf, position, size, scale); + + existing.accumulateSum(addend); + } + } + + @Override + public float getFloat(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not support getFloat()"); + } + + @Override + public long getLong(ByteBuffer buf, int position) + { + throw new UnsupportedOperationException(getClass().getSimpleName() + " does not support getLong()"); + } + + @Override + public void close() + { + // no resources to cleanup + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregator.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregator.java new file mode 100644 index 00000000000..5f97f6f6181 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregator.java @@ -0,0 +1,30 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalSumSqlAggregator extends CompressedBigDecimalSqlAggregatorBase +{ + static final String NAME = "BIG_SUM"; + + public CompressedBigDecimalSumSqlAggregator() + { + super(NAME, CompressedBigDecimalSumAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/Utils.java b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/Utils.java index 97eb5bd615f..a214bfd057a 100644 --- a/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/Utils.java +++ b/extensions-contrib/compressed-bigdecimal/src/main/java/org/apache/druid/compressedbigdecimal/Utils.java @@ -19,9 +19,10 @@ package org.apache.druid.compressedbigdecimal; +import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.data.IndexedInts; +import javax.annotation.Nullable; import java.math.BigDecimal; import java.math.RoundingMode; import java.nio.ByteBuffer; @@ -43,11 +44,11 @@ public class Utils * @param rhs The object to accumulate * @return a reference to this */ - public static CompressedBigDecimal accumulate(CompressedBigDecimal lhs, BigDecimal rhs) + public static CompressedBigDecimal accumulateSum(CompressedBigDecimal lhs, BigDecimal rhs) { CompressedBigDecimal abd = - new ArrayCompressedBigDecimal(rhs.setScale(lhs.getScale())); - return lhs.accumulate(abd); + new ArrayCompressedBigDecimal(rhs.setScale(lhs.getScale(), RoundingMode.HALF_UP)); + return lhs.accumulateSum(abd); } /** @@ -63,29 +64,10 @@ public class Utils * @param rhsScale The scale to apply to the long being accumulated * @return a reference to this */ - public static CompressedBigDecimal accumulate(CompressedBigDecimal lhs, long rhs, int rhsScale) + public static CompressedBigDecimal accumulateSum(CompressedBigDecimal lhs, long rhs, int rhsScale) { CompressedBigDecimal abd = new ArrayCompressedBigDecimal(rhs, rhsScale); - return lhs.accumulate(abd); - } - - /** - * Accumulate using IndexedInts read from Druid's segment file. - * - * @param lhs The object into which to accumulate - * @param rhs IndexedInts representing array of magnitude values - * @param rhsScale the scale - * @return a reference to this - */ - public static CompressedBigDecimal accumulate(CompressedBigDecimal lhs, IndexedInts rhs, int rhsScale) - { - if (rhs.size() > lhs.getArraySize()) { - throw new IllegalArgumentException("Right hand side too big to fit in the result value"); - } - CompressedBigDecimal.internalAdd(lhs.getArraySize(), lhs, CompressedBigDecimal::getArrayEntry, - CompressedBigDecimal::setArrayEntry, rhs.size(), rhs, IndexedInts::get - ); - return lhs; + return lhs.accumulateSum(abd); } /** @@ -97,15 +79,18 @@ public class Utils * @param lhsScale The scale of the left * @param rhs the right side to accumlate */ - public static void accumulate(ByteBuffer buf, int pos, int lhsSize, int lhsScale, CompressedBigDecimal rhs) + public static void accumulateSum(ByteBuffer buf, int pos, int lhsSize, int lhsScale, CompressedBigDecimal rhs) { - if (rhs.getArraySize() > lhsSize) { - throw new IllegalArgumentException("Right hand side too big to fit in the result value"); - } + Preconditions.checkArgument( + rhs.getScale() == lhsScale, + "scales do not match: lhs [%s] vs rhs [%s]", + lhsScale, + rhs.getScale() + ); + Preconditions.checkArgument(rhs.getArraySize() <= lhsSize, "Right hand side too big to fit in the result value"); + BufferAccessor accessor = BufferAccessor.prepare(pos); - if (rhs.getScale() != lhsScale) { - rhs = Utils.scale(rhs, lhsScale); - } + CompressedBigDecimal.internalAdd( lhsSize, buf, @@ -117,6 +102,15 @@ public class Utils ); } + public static CompressedBigDecimal scaleIfNeeded(CompressedBigDecimal val, int scale) + { + if (val.getScale() != scale) { + return new ArrayCompressedBigDecimal(val.toBigDecimal().setScale(scale, RoundingMode.UP)); + } else { + return val; + } + } + public static CompressedBigDecimal scale(CompressedBigDecimal val, int scale) { return new ArrayCompressedBigDecimal(val.toBigDecimal().setScale(scale, RoundingMode.UP)); @@ -127,21 +121,28 @@ public class Utils return objToCompressedBigDecimal(obj, false); } + @Nullable + public static CompressedBigDecimal objToCompressedBigDecimalWithScale( + Object obj, + int scale, + boolean strictNumberParse + ) + { + CompressedBigDecimal compressedBigDecimal = Utils.objToCompressedBigDecimal(obj, strictNumberParse); + + if (compressedBigDecimal != null) { + return scaleIfNeeded(compressedBigDecimal, scale); + } else { + return null; + } + } + public static CompressedBigDecimal objToCompressedBigDecimal(Object obj, boolean strictNumberParse) { CompressedBigDecimal result; + if (obj == null) { result = null; - } else if (obj instanceof BigDecimal) { - result = new ArrayCompressedBigDecimal((BigDecimal) obj); - } else if (obj instanceof Long) { - result = new ArrayCompressedBigDecimal(new BigDecimal((Long) obj)); - } else if (obj instanceof Integer) { - result = new ArrayCompressedBigDecimal(new BigDecimal((Integer) obj)); - } else if (obj instanceof Double) { - result = new ArrayCompressedBigDecimal(BigDecimal.valueOf((Double) obj)); - } else if (obj instanceof Float) { - result = new ArrayCompressedBigDecimal(BigDecimal.valueOf((Float) obj)); } else if (obj instanceof String) { try { result = new ArrayCompressedBigDecimal(new BigDecimal((String) obj)); @@ -153,10 +154,20 @@ public class Utils result = new ArrayCompressedBigDecimal(0L, 0); } } + } else if (obj instanceof BigDecimal) { + result = new ArrayCompressedBigDecimal((BigDecimal) obj); + } else if (obj instanceof Long) { + result = new ArrayCompressedBigDecimal(new BigDecimal((Long) obj)); + } else if (obj instanceof Integer) { + result = new ArrayCompressedBigDecimal(new BigDecimal((Integer) obj)); + } else if (obj instanceof Double) { + result = new ArrayCompressedBigDecimal(BigDecimal.valueOf((Double) obj)); + } else if (obj instanceof Float) { + result = new ArrayCompressedBigDecimal(BigDecimal.valueOf((Float) obj)); } else if (obj instanceof CompressedBigDecimal) { result = (CompressedBigDecimal) obj; } else { - throw new ISE("Unknown extraction value type: [%s]", obj.getClass().getSimpleName()); + throw new ISE("Unknown value type: [%s]", obj.getClass().getName()); } return result; @@ -164,7 +175,7 @@ public class Utils /** * Helper class that maintains a cache of thread local objects that can be used to access - * a ByteBuffer in {@link Utils#accumulate(ByteBuffer, int, int, int, CompressedBigDecimal)}. + * a ByteBuffer in {@link Utils#accumulateSum(ByteBuffer, int, int, int, CompressedBigDecimal)}. */ private static class BufferAccessor implements ToIntBiFunction, ObjBiIntConsumer { diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/AggregatorCombinerFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/AggregatorCombinerFactoryTest.java deleted file mode 100644 index 67c9904b2e3..00000000000 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/AggregatorCombinerFactoryTest.java +++ /dev/null @@ -1,203 +0,0 @@ -/* - * 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. 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 - * - * 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 org.apache.druid.compressedbigdecimal; - -import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.data.ColumnarInts; -import org.apache.druid.segment.data.ColumnarMultiInts; -import org.apache.druid.segment.data.ReadableOffset; -import org.easymock.EasyMock; -import org.junit.Assert; -import org.junit.Test; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.Arrays; - - -public class AggregatorCombinerFactoryTest -{ - /** - * Test method for {@link CompressedBigDecimalColumn}. - */ - @Test - public void testCompressedBigDecimalColumn() - { - ColumnarMultiInts cmi = EasyMock.createMock(ColumnarMultiInts.class); - ColumnarInts ci = EasyMock.createMock(ColumnarInts.class); - ReadableOffset ro = EasyMock.createMock(ReadableOffset.class); - CompressedBigDecimalColumn cbr = new CompressedBigDecimalColumn(ci, cmi); - Assert.assertEquals(CompressedBigDecimalModule.COMPRESSED_BIG_DECIMAL, cbr.getTypeName()); - Assert.assertEquals(0, cbr.getLength()); - Assert.assertEquals(CompressedBigDecimalColumn.class, cbr.getClazz()); - Assert.assertNotNull(cbr.makeColumnValueSelector(ro)); - } - - /** - * Test method for {@link CompressedBigDecimalAggregatorFactory}. - */ - @Test - public void testCompressedBigDecimalAggregatorFactory() - { - CompressedBigDecimalAggregatorFactory cf = new CompressedBigDecimalAggregatorFactory( - "name", - "fieldName", - 9, - 0, - false - ); - Assert.assertEquals( - "CompressedBigDecimalSumAggregatorFactory{name='name', type='compressedBigDecimal', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}", - cf.toString() - ); - Assert.assertNotNull(cf.getCacheKey()); - Assert.assertNull(cf.deserialize(null)); - Assert.assertEquals("5", cf.deserialize(new BigDecimal(5)).toString()); - Assert.assertEquals("5", cf.deserialize(5d).toString()); - Assert.assertEquals("5", cf.deserialize("5").toString()); - Assert.assertEquals( - "[CompressedBigDecimalSumAggregatorFactory{name='fieldName', type='compressedBigDecimal', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", - Arrays.toString(cf.getRequiredColumns().toArray()) - ); - Assert.assertEquals("0", cf.combine(null, null).toString()); - Assert.assertEquals("4", cf.combine(new BigDecimal(4), null).toString()); - Assert.assertEquals("4", cf.combine(null, new BigDecimal(4)).toString()); - Assert.assertEquals( - "8", - cf.combine( - new ArrayCompressedBigDecimal(new BigDecimal(4)), - new ArrayCompressedBigDecimal(new BigDecimal(4)) - ).toString() - ); - } - - /** - * Test method for {@link CompressedBigDecimalAggregatorFactory#deserialize(Object)}. - */ - @Test(expected = RuntimeException.class) - public void testCompressedBigDecimalAggregatorFactoryDeserialize() - { - CompressedBigDecimalAggregatorFactory cf = new CompressedBigDecimalAggregatorFactory( - "name", - "fieldName", - 9, - 0, - false - ); - cf.deserialize(5); - } - - /** - * Test method for {@link CompressedBigDecimalBufferAggregator#getFloat(ByteBuffer, int)} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalBufferAggregatorGetFloat() - { - ColumnValueSelector cs = EasyMock.createMock(ColumnValueSelector.class); - ByteBuffer bbuf = ByteBuffer.allocate(10); - CompressedBigDecimalBufferAggregator ca = new CompressedBigDecimalBufferAggregator(4, 0, cs, false); - ca.getFloat(bbuf, 0); - } - - /** - * Test method for {@link CompressedBigDecimalBufferAggregator#getLong(ByteBuffer, int)} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalBufferAggregatorGetLong() - { - ColumnValueSelector cs = EasyMock.createMock(ColumnValueSelector.class); - ByteBuffer bbuf = ByteBuffer.allocate(10); - CompressedBigDecimalBufferAggregator ca = new CompressedBigDecimalBufferAggregator(4, 0, cs, false); - ca.getLong(bbuf, 0); - } - - /** - * Test method for {@link CompressedBigDecimalAggregateCombiner#getObject()} - */ - @Test - public void testCompressedBigDecimalAggregateCombinerGetObject() - { - CompressedBigDecimalAggregateCombiner cc = new CompressedBigDecimalAggregateCombiner(); - CompressedBigDecimal c = cc.getObject(); - Assert.assertSame(null, c); - } - - /** - * Test method for {@link CompressedBigDecimalAggregateCombiner#getClass()} - */ - @Test - public void testCompressedBigDecimalAggregateCombinerClassofObject() - { - CompressedBigDecimalAggregateCombiner cc = new CompressedBigDecimalAggregateCombiner(); - Assert.assertSame(CompressedBigDecimalAggregateCombiner.class, cc.getClass()); - } - - /** - * Test method for {@link CompressedBigDecimalAggregateCombiner#getLong()} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalAggregateCombinerGetLong() - { - CompressedBigDecimalAggregateCombiner cc = new CompressedBigDecimalAggregateCombiner(); - cc.getLong(); - } - - /** - * Test method for {@link CompressedBigDecimalAggregateCombiner#getFloat()} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalAggregateCombinerGetFloat() - { - CompressedBigDecimalAggregateCombiner cc = new CompressedBigDecimalAggregateCombiner(); - cc.getFloat(); - } - - /** - * Test method for {@link CompressedBigDecimalAggregateCombiner#getDouble()} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalAggregateCombinerGetDouble() - { - CompressedBigDecimalAggregateCombiner cc = new CompressedBigDecimalAggregateCombiner(); - cc.getDouble(); - } - - /** - * Test method for {@link CompressedBigDecimalAggregator#getFloat()} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalAggregatorGetFloat() - { - ColumnValueSelector cv = EasyMock.createMock(ColumnValueSelector.class); - CompressedBigDecimalAggregator cc = new CompressedBigDecimalAggregator(2, 0, cv, false); - cc.getFloat(); - } - - /** - * Test method for {@link CompressedBigDecimalAggregator#getLong()} - */ - @Test(expected = UnsupportedOperationException.class) - public void testCompressedBigDecimalAggregatorGetLong() - { - ColumnValueSelector cv = EasyMock.createMock(ColumnValueSelector.class); - CompressedBigDecimalAggregator cc = new CompressedBigDecimalAggregator(2, 0, cv, false); - cc.getLong(); - } -} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimalTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimalTest.java index 3d61d078d73..bb69b2b019d 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimalTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ArrayCompressedBigDecimalTest.java @@ -27,7 +27,6 @@ import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; -import static org.apache.druid.compressedbigdecimal.Utils.accumulate; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotSame; @@ -144,32 +143,32 @@ public class ArrayCompressedBigDecimalTest @Test public void testCompareToOptimization() { - ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); - ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); - ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE - ); - ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); + ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE + ); // leave this as 0 - ArrayCompressedBigDecimal d5 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d5 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); - d1.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)), d1.getScale())); - d2.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(-2L * Integer.MAX_VALUE)), d2.getScale())); - d3.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d3.getScale())); - d4.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d4.getScale())); + d1.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)), d1.getScale())); + d2.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(-2L * Integer.MAX_VALUE)), d2.getScale())); + d3.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d3.getScale())); + d4.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d4.getScale())); Assert.assertEquals(-1, d2.compareTo(d1, true)); Assert.assertEquals(1, d1.compareTo(d2, true)); @@ -189,14 +188,14 @@ public class ArrayCompressedBigDecimalTest @Test(expected = IAE.class) public void testCompareToFailOptimized() { - ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE - ); - ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + 1 + ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); + ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE + 1 + ); // size doesn't match, so an IAE will be thrown d1.compareTo(d2, true); @@ -205,32 +204,32 @@ public class ArrayCompressedBigDecimalTest @Test public void testCompareToOptimizedUnequalSizes() { - ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 1, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d1 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 1, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); - ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 2, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE - ); - ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 3, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE - ); - ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 4, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 2, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE + ); + ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 3, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE + ); + ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 4, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); // leave this as 0 - ArrayCompressedBigDecimal d5 = ArrayCompressedBigDecimal.allocate( - CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 5, - CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE + ArrayCompressedBigDecimal d5 = ArrayCompressedBigDecimal.allocateZero( + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 5, + CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE ); - d1.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)), d1.getScale())); - d2.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(-8L * Integer.MAX_VALUE)), d2.getScale())); - d3.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d3.getScale())); - d4.accumulate(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d4.getScale())); + d1.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)), d1.getScale())); + d2.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(-8L * Integer.MAX_VALUE)), d2.getScale())); + d3.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d3.getScale())); + d4.accumulateSum(Utils.scale(new ArrayCompressedBigDecimal(new BigDecimal(8L * Integer.MAX_VALUE)), d4.getScale())); Assert.assertEquals(-1, d2.compareTo(d1, true)); Assert.assertEquals(1, d1.compareTo(d2, true)); @@ -353,33 +352,33 @@ public class ArrayCompressedBigDecimalTest } /** - * Test method for {@link ArrayCompressedBigDecimal#allocate(int, int)}. + * Test method for {@link ArrayCompressedBigDecimal#allocateZero(int, int)}. */ @Test public void testAllocate() { - ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocate(2, 5); + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 5); assertEquals(5, bd.getScale()); assertEquals(2, bd.getArray().length); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test public void testSimpleAccumulate() { - ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocate(2, 0); + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 0); ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{0x00000001, 0}, 0); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{1, 0}, bd.getArray()); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{2, 0}, bd.getArray()); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test public void testSimpleAccumulateOverflow() @@ -387,14 +386,14 @@ public class ArrayCompressedBigDecimalTest ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.wrap(new int[]{0x80000000, 0}, 0); ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{0x7fffffff, 0}, 0); ArrayCompressedBigDecimal add1 = ArrayCompressedBigDecimal.wrap(new int[]{0x00000001, 0}, 0); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{0xffffffff, 0}, bd.getArray()); - bd.accumulate(add1); + bd.accumulateSum(add1); assertArrayEquals(new int[]{0, 1}, bd.getArray()); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test public void testSimpleAccumulateUnderflow() @@ -403,12 +402,12 @@ public class ArrayCompressedBigDecimalTest ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{-1, -1}, 0); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{0xffffffff, 0}, bd.getArray()); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test public void testUnevenAccumulateUnderflow() @@ -417,12 +416,12 @@ public class ArrayCompressedBigDecimalTest ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{-1}, 0); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{0xffffffff, 0}, bd.getArray()); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test public void testUnevenAccumulateOverflow() @@ -431,12 +430,12 @@ public class ArrayCompressedBigDecimalTest ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{1}, 0); - bd.accumulate(add); + bd.accumulateSum(add); assertArrayEquals(new int[]{0, 2}, bd.getArray()); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test(expected = IllegalArgumentException.class) public void testUnevenAccumulateOverflowWithTruncate() @@ -445,18 +444,18 @@ public class ArrayCompressedBigDecimalTest ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{1, 1}, 0); - bd.accumulate(add); + bd.accumulateSum(add); } /** - * Test method for {@link ArrayCompressedBigDecimal#accumulate(CompressedBigDecimal)}. + * Test method for {@link ArrayCompressedBigDecimal#accumulateSum(CompressedBigDecimal)}. */ @Test(expected = IllegalArgumentException.class) - public void testAccumulateScaleMismatch() + public void testAccumulateSumScaleMismatch() { - ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocate(2, 1); + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 1); ArrayCompressedBigDecimal add = new ArrayCompressedBigDecimal(1, 0); - bd.accumulate(add); + bd.accumulateSum(add); } /** @@ -522,38 +521,38 @@ public class ArrayCompressedBigDecimalTest } /** - * Test method for {@link Utils#accumulate(ByteBuffer, int, int, int, CompressedBigDecimal)} + * Test method for {@link Utils#accumulateSum(ByteBuffer, int, int, int, CompressedBigDecimal)} */ @Test(expected = IllegalArgumentException.class) - public void testUtilsAccumulateByteBuf() + public void testUtilsAccumulateSumByteBuf() { ByteBuffer buf = ByteBuffer.allocate(4); - accumulate(buf, 0, 1, 2, new ArrayCompressedBigDecimal(new BigDecimal(Long.MAX_VALUE))); + Utils.accumulateSum(buf, 0, 1, 2, new ArrayCompressedBigDecimal(new BigDecimal(Long.MAX_VALUE))); } /** - * Test method for {@link Utils#accumulate(CompressedBigDecimal, long, int)} + * Test method for {@link Utils#accumulateSum(CompressedBigDecimal, long, int)} */ @Test(expected = IllegalArgumentException.class) - public void testUtilsAccumulateCbdWithExeception() + public void testUtilsAccumulateSumCbdWithExeception() { BigDecimal bd = new BigDecimal(new BigInteger("1")); ArrayCompressedBigDecimal d = new ArrayCompressedBigDecimal(bd); - accumulate(d, 0L, 1); + Utils.accumulateSum(d, 0L, 1); } /** - * Test method for {@link Utils#accumulate(CompressedBigDecimal, long, int)} + * Test method for {@link Utils#accumulateSum(CompressedBigDecimal, long, int)} */ @Test - public void testUtilsAccumulateCbd() + public void testUtilsAccumulateSumCbd() { - ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocate(2, 0); + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 0); ArrayCompressedBigDecimal add = ArrayCompressedBigDecimal.wrap(new int[]{0x00000001, 0}, 0); - bd.accumulate(add); - accumulate(bd, 1, 0); + bd.accumulateSum(add); + Utils.accumulateSum(bd, 1, 0); assertEquals("2", bd.toString()); - CompressedBigDecimal x = accumulate(bd, new BigDecimal("2")); + CompressedBigDecimal x = Utils.accumulateSum(bd, new BigDecimal("2")); assertEquals(4, x.intValue()); CompressedBigDecimalObjectStrategy c1 = new CompressedBigDecimalObjectStrategy(); @@ -588,4 +587,74 @@ public class ArrayCompressedBigDecimalTest assertEquals(0, c1.compare(bd, acd)); assertEquals(0, cbd.intValue()); } + + @Test + public void testSetMinValue() + { + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 0); + + bd.setMinValue(); + Assert.assertEquals("-9223372036854775808", bd.toString()); + } + + @Test + public void testSetMaxValue() + { + ArrayCompressedBigDecimal bd = ArrayCompressedBigDecimal.allocateZero(2, 0); + + bd.setMaxValue(); + Assert.assertEquals("9223372036854775807", bd.toString()); + } + + @Test + public void testAccumulateMaxInteger() + { + ArrayCompressedBigDecimal bdBase = ArrayCompressedBigDecimal.allocateZero(6, 0); + bdBase.setMinValue(); + ArrayCompressedBigDecimal bd1 = new ArrayCompressedBigDecimal(new BigDecimal(Long.MIN_VALUE)); + ArrayCompressedBigDecimal bd2 = new ArrayCompressedBigDecimal(new BigDecimal("-1")); + ArrayCompressedBigDecimal bd3 = new ArrayCompressedBigDecimal(new BigDecimal(Long.MAX_VALUE)); + ArrayCompressedBigDecimal bd4 = new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)); + + bdBase.accumulateMax(bd1); + Assert.assertEquals(Long.toString(Long.MIN_VALUE), bdBase.toString()); + bdBase.accumulateMax(bd2); + Assert.assertEquals("-1", bdBase.toString()); + bdBase.accumulateMax(bd3); + Assert.assertEquals(Long.toString(Long.MAX_VALUE), bdBase.toString()); + bdBase.accumulateMax(bd4); + Assert.assertEquals(Long.toString(Long.MAX_VALUE), bdBase.toString()); + } + + @Test + public void testAccumulateMinInteger() + { + ArrayCompressedBigDecimal bdBase = ArrayCompressedBigDecimal.allocateZero(6, 0); + bdBase.setMaxValue(); + + ArrayCompressedBigDecimal bd1 = new ArrayCompressedBigDecimal(new BigDecimal("-1")); + ArrayCompressedBigDecimal bd2 = new ArrayCompressedBigDecimal(new BigDecimal(Long.MAX_VALUE)); + ArrayCompressedBigDecimal bd3 = new ArrayCompressedBigDecimal(new BigDecimal(Integer.MAX_VALUE)); + ArrayCompressedBigDecimal bd4 = new ArrayCompressedBigDecimal(new BigDecimal(Integer.MIN_VALUE)); + ArrayCompressedBigDecimal bd5 = new ArrayCompressedBigDecimal(new BigDecimal(Long.MIN_VALUE)); + ArrayCompressedBigDecimal bd6 = ArrayCompressedBigDecimal.wrap(new int[]{0, 0, 0, 0, 0, 0x80000000}, 0); + ArrayCompressedBigDecimal bd7 = ArrayCompressedBigDecimal.wrap(new int[]{0, 0, 0, 0, 0, 1}, 0); + + bdBase.accumulateMin(bd1); + Assert.assertEquals("-1", bdBase.toString()); + bdBase.accumulateMin(bd2); + Assert.assertEquals("-1", bdBase.toString()); + bdBase.accumulateMin(bd3); + Assert.assertEquals("-1", bdBase.toString()); + bdBase.accumulateMin(bd4); + Assert.assertEquals(Integer.toString(Integer.MIN_VALUE), bdBase.toString()); + bdBase.accumulateMin(bd5); + Assert.assertEquals(Long.toString(Long.MIN_VALUE), bdBase.toString()); + // smallest possible 6-int[] number + bdBase.accumulateMin(bd6); + Assert.assertEquals("-3138550867693340381917894711603833208051177722232017256448", bdBase.toString()); + // this tests that we don't do the subtraction and overflow + bdBase.accumulateMin(bd7); + Assert.assertEquals("-3138550867693340381917894711603833208051177722232017256448", bdBase.toString()); + } } diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimalTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimalTest.java new file mode 100644 index 00000000000..dfa1c1b4604 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/ByteBufferCompressedBigDecimalTest.java @@ -0,0 +1,88 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.junit.Assert; +import org.junit.Test; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; + +public class ByteBufferCompressedBigDecimalTest +{ + private static final int SIZE = 4; + private static final int SCALE = 2; + + private final ByteBuffer byteBuffer = ByteBuffer.allocate(100); + + @Test + public void testInitZero() + { + ByteBufferCompressedBigDecimal.initZero(byteBuffer, 0, SIZE); + + testBufferValue("0"); + } + + @Test + public void testInitMin() + { + ByteBufferCompressedBigDecimal.initMin(byteBuffer, 0, SIZE); + + testBufferValue("-1701411834604692317316873037158841057.28"); + } + + @Test + public void testInitMax() + { + ByteBufferCompressedBigDecimal.initMax(byteBuffer, 0, SIZE); + + testBufferValue("1701411834604692317316873037158841057.27"); + } + + @Test + public void testSetValue() + { + ByteBufferCompressedBigDecimal.initZero(byteBuffer, 0, SIZE); + ByteBufferCompressedBigDecimal compressedBigDecimal = new ByteBufferCompressedBigDecimal( + byteBuffer, + 0, + SIZE, + SCALE + ); + String value = "10500.63"; + + compressedBigDecimal.setValue(new ArrayCompressedBigDecimal(new BigDecimal(value))); + + testBufferValue(value); + + } + + public void testBufferValue(String expectedValue) + { + ByteBufferCompressedBigDecimal compressedBigDecimal = new ByteBufferCompressedBigDecimal( + byteBuffer, + 0, + SIZE, + SCALE + ); + + Assert.assertEquals(expectedValue, compressedBigDecimal.toString()); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTestBase.java similarity index 79% rename from extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTest.java rename to extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTestBase.java index fb044c156d3..13d7ac34306 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorGroupByTestBase.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; -import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; import org.hamcrest.collection.IsCollectionWithSize; import org.hamcrest.collection.IsMapContaining; @@ -46,51 +45,38 @@ import java.io.File; import java.io.IOException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.TimeZone; -/** - * Unit tests for AccumulatingDecimalAggregator. - */ @RunWith(Parameterized.class) -public class CompressedBigDecimalAggregatorGroupByTest +public abstract class CompressedBigDecimalAggregatorGroupByTestBase { private final AggregationTestHelper helper; @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(new File("target")); + private final CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig; + /** * Constructor. * * @param config config object */ - public CompressedBigDecimalAggregatorGroupByTest(GroupByQueryConfig config) + public CompressedBigDecimalAggregatorGroupByTestBase( + GroupByQueryConfig config, + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig + ) { + this.cbdGroupByQueryConfig = cbdGroupByQueryConfig; CompressedBigDecimalModule module = new CompressedBigDecimalModule(); CompressedBigDecimalModule.registerSerde(); helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( module.getJacksonModules(), config, tempFolder); } - /** - * Constructor feeder. - * - * @return constructors - */ - @Parameterized.Parameters(name = "{0}") - public static Collection constructorFeeder() - { - final List constructors = new ArrayList<>(); - for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - constructors.add(new Object[]{config}); - } - return constructors; - } /** * Default setup of UTC timezone. @@ -112,7 +98,7 @@ public class CompressedBigDecimalAggregatorGroupByTest { String groupByQueryJson = Resources.asCharSource( - this.getClass().getResource("/" + "bd_test_groupby_query.json"), + this.getClass().getResource("/" + cbdGroupByQueryConfig.getJsonQueryFile()), StandardCharsets.UTF_8 ).read(); @@ -124,7 +110,7 @@ public class CompressedBigDecimalAggregatorGroupByTest StandardCharsets.UTF_8 ).read(), Resources.asCharSource( - this.getClass().getResource("/" + "bd_test_aggregators.json"), + this.getClass().getResource("/" + cbdGroupByQueryConfig.getJsonAggregatorsFile()), StandardCharsets.UTF_8 ).read(), 0, @@ -147,17 +133,26 @@ public class CompressedBigDecimalAggregatorGroupByTest Assert.assertThat(event, IsMapWithSize.aMapWithSize(3)); Assert.assertThat( event, - IsMapContaining.hasEntry("cbdRevenueFromString", new BigDecimal("15000000010.000000005")) + IsMapContaining.hasEntry( + "cbdRevenueFromString", + new ArrayCompressedBigDecimal(new BigDecimal(cbdGroupByQueryConfig.getStringRevenue())) + ) ); // long conversion of 5000000000.000000005 results in null/0 value Assert.assertThat( event, - IsMapContaining.hasEntry("cbdRevenueFromLong", new BigDecimal("10000000010.000000000")) + IsMapContaining.hasEntry( + "cbdRevenueFromLong", + new ArrayCompressedBigDecimal(new BigDecimal(cbdGroupByQueryConfig.getLongRevenue())) + ) ); // double input changes 5000000000.000000005 to 5000000000.5 to fit in double mantissa space Assert.assertThat( event, - IsMapContaining.hasEntry("cbdRevenueFromDouble", new BigDecimal("15000000010.500000000")) + IsMapContaining.hasEntry( + "cbdRevenueFromDouble", + new ArrayCompressedBigDecimal(new BigDecimal(cbdGroupByQueryConfig.getDoubleRevenue())) + ) ); } } diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTestBase.java new file mode 100644 index 00000000000..8ae349605ec --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTestBase.java @@ -0,0 +1,35 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.aggregation.Aggregator; +import org.junit.Assert; +import org.junit.Test; + +public abstract class CompressedBigDecimalAggregatorTestBase +{ + @Test + public abstract void testNoData(); + + protected void testNoDataHelper(Aggregator aggregator, Object expectedValue) + { + Assert.assertEquals(expectedValue, aggregator.get()); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTest.java deleted file mode 100644 index 15c38ab0a47..00000000000 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTest.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * 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. 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 - * - * 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 org.apache.druid.compressedbigdecimal; - -import com.google.common.collect.Iterables; -import com.google.common.io.Resources; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.query.Result; -import org.apache.druid.query.aggregation.AggregationTestHelper; -import org.apache.druid.query.timeseries.TimeseriesResultValue; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.Map; -import java.util.TimeZone; - -import static org.hamcrest.collection.IsMapContaining.hasEntry; -import static org.hamcrest.collection.IsMapWithSize.aMapWithSize; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; - -/** - * Unit tests for AccumulatingDecimalAggregator. - */ -public class CompressedBigDecimalAggregatorTimeseriesTest -{ - private final AggregationTestHelper helper; - - static { - NullHandling.initializeForTests(); - } - - @Rule - public final TemporaryFolder tempFolder = new TemporaryFolder(new File("target")); - - /** - * Constructor. - * * */ - public CompressedBigDecimalAggregatorTimeseriesTest() - { - CompressedBigDecimalModule module = new CompressedBigDecimalModule(); - CompressedBigDecimalModule.registerSerde(); - helper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( - module.getJacksonModules(), tempFolder); - } - - /** - * Default setup of UTC timezone. - */ - @BeforeClass - public static void setupClass() - { - System.setProperty("user.timezone", "UTC"); - } - - /** - * ingetion method for all timeseries query. - * - * @throws IOException IOException - * @throws Exception Exception - */ - @Test - public void testIngestAndTimeseriesQuery() throws IOException, Exception - { - Sequence seq = helper.createIndexAndRunQueryOnSegment( - this.getClass().getResourceAsStream("/" + "bd_test_data.csv"), - Resources.asCharSource(getClass().getResource( - "/" + "bd_test_data_parser.json"), - StandardCharsets.UTF_8 - ).read(), - Resources.asCharSource( - this.getClass().getResource("/" + "bd_test_aggregators.json"), - StandardCharsets.UTF_8 - ).read(), - 0, - Granularities.NONE, - 5, - Resources.asCharSource( - this.getClass().getResource("/" + "bd_test_timeseries_query.json"), - StandardCharsets.UTF_8 - ).read() - ); - - TimeseriesResultValue result = ((Result) Iterables.getOnlyElement(seq.toList())).getValue(); - Map event = result.getBaseObject(); - assertEquals(new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))), - ((Result) Iterables.getOnlyElement(seq.toList())).getTimestamp()); - assertThat(event, aMapWithSize(1)); - assertThat(event, hasEntry("revenue", new BigDecimal("15000000010.000000005"))); - } - - /** - * Test using multiple segments. - * - * @throws Exception an exception - */ - @Test - public void testIngestMultipleSegmentsAndTimeseriesQuery() throws Exception - { - File segmentDir1 = tempFolder.newFolder(); - helper.createIndex( - new File(this.getClass().getResource("/" + "bd_test_data.csv").getFile()), - Resources.asCharSource(this.getClass().getResource("/" + "bd_test_data_parser.json"), - StandardCharsets.UTF_8).read(), - Resources.asCharSource(this.getClass().getResource("/" + "bd_test_aggregators.json"), - StandardCharsets.UTF_8).read(), - segmentDir1, - 0, - Granularities.NONE, - 5); - File segmentDir2 = tempFolder.newFolder(); - helper.createIndex( - new File(this.getClass().getResource("/" + "bd_test_zero_data.csv").getFile()), - Resources.asCharSource(this.getClass().getResource("/" + "bd_test_data_parser.json"), - StandardCharsets.UTF_8).read(), - Resources.asCharSource(this.getClass().getResource("/" + "bd_test_aggregators.json"), - StandardCharsets.UTF_8).read(), - segmentDir2, - 0, - Granularities.NONE, - 5); - - Sequence seq = helper.runQueryOnSegments( - Arrays.asList(segmentDir1, segmentDir2), - Resources.asCharSource( - this.getClass().getResource("/" + "bd_test_timeseries_query.json"), - StandardCharsets.UTF_8 - ).read()); - - TimeseriesResultValue result = ((Result) Iterables.getOnlyElement(seq.toList())).getValue(); - Map event = result.getBaseObject(); - assertEquals(new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))), - ((Result) Iterables.getOnlyElement(seq.toList())).getTimestamp()); - assertThat(event, aMapWithSize(1)); - assertThat(event, hasEntry("revenue", new BigDecimal("15000000010.000000005"))); - - } -} - diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTestBase.java new file mode 100644 index 00000000000..2e7c5111847 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalAggregatorTimeseriesTestBase.java @@ -0,0 +1,193 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.google.common.collect.Iterables; +import com.google.common.io.Resources; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregationTestHelper; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Map; +import java.util.TimeZone; + +import static org.hamcrest.collection.IsMapContaining.hasEntry; +import static org.hamcrest.collection.IsMapWithSize.aMapWithSize; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +public abstract class CompressedBigDecimalAggregatorTimeseriesTestBase +{ + private final AggregationTestHelper helper; + + static { + NullHandling.initializeForTests(); + } + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(new File("target")); + + /** + * Constructor. + * * + */ + public CompressedBigDecimalAggregatorTimeseriesTestBase() + { + CompressedBigDecimalModule module = new CompressedBigDecimalModule(); + CompressedBigDecimalModule.registerSerde(); + helper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( + module.getJacksonModules(), tempFolder); + } + + /** + * Default setup of UTC timezone. + */ + @BeforeClass + public static void setupClass() + { + System.setProperty("user.timezone", "UTC"); + } + + + @Test + public abstract void testIngestAndTimeseriesQuery() throws Exception; + + protected void testIngestAndTimeseriesQueryHelper( + String jsonAggregatorsFile, + String jsonQueryFile, + String expected + ) throws Exception + { + Sequence seq = helper.createIndexAndRunQueryOnSegment( + this.getClass().getResourceAsStream("/" + "bd_test_data.csv"), + Resources.asCharSource( + getClass().getResource( + "/" + "bd_test_data_parser.json"), + StandardCharsets.UTF_8 + ).read(), + Resources.asCharSource( + this.getClass().getResource("/" + jsonAggregatorsFile), + StandardCharsets.UTF_8 + ).read(), + 0, + Granularities.NONE, + 5, + Resources.asCharSource( + this.getClass().getResource("/" + jsonQueryFile), + StandardCharsets.UTF_8 + ).read() + ); + + TimeseriesResultValue result = ((Result) Iterables.getOnlyElement(seq.toList())).getValue(); + Map event = result.getBaseObject(); + assertEquals( + new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))), + ((Result) Iterables.getOnlyElement(seq.toList())).getTimestamp() + ); + assertThat(event, aMapWithSize(1)); + assertThat( + event, + hasEntry("cbdStringRevenue", new ArrayCompressedBigDecimal(new BigDecimal(expected))) + ); + } + + /** + * Test using multiple segments. + * + * @throws Exception an exception + */ + @Test + public abstract void testIngestMultipleSegmentsAndTimeseriesQuery() throws Exception; + + protected void testIngestMultipleSegmentsAndTimeseriesQueryHelper( + String jsonAggregatorsFile, + String jsonQueryFile, + String expected + ) throws Exception + { + File segmentDir1 = tempFolder.newFolder(); + helper.createIndex( + new File(this.getClass().getResource("/" + "bd_test_data.csv").getFile()), + Resources.asCharSource( + this.getClass().getResource("/" + "bd_test_data_parser.json"), + StandardCharsets.UTF_8 + ).read(), + Resources.asCharSource( + this.getClass().getResource("/" + jsonAggregatorsFile), + StandardCharsets.UTF_8 + ).read(), + segmentDir1, + 0, + Granularities.NONE, + 5 + ); + File segmentDir2 = tempFolder.newFolder(); + helper.createIndex( + new File(this.getClass().getResource("/" + "bd_test_zero_data.csv").getFile()), + Resources.asCharSource( + this.getClass().getResource("/" + "bd_test_data_parser.json"), + StandardCharsets.UTF_8 + ).read(), + Resources.asCharSource( + this.getClass().getResource("/" + jsonAggregatorsFile), + StandardCharsets.UTF_8 + ).read(), + segmentDir2, + 0, + Granularities.NONE, + 5 + ); + + Sequence seq = helper.runQueryOnSegments( + Arrays.asList(segmentDir1, segmentDir2), + Resources.asCharSource( + this.getClass().getResource("/" + jsonQueryFile), + StandardCharsets.UTF_8 + ).read() + ); + + TimeseriesResultValue result = ((Result) Iterables.getOnlyElement(seq.toList())).getValue(); + Map event = result.getBaseObject(); + assertEquals( + new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))), + ((Result) Iterables.getOnlyElement(seq.toList())).getTimestamp() + ); + assertThat(event, aMapWithSize(1)); + assertThat( + event, + hasEntry("cbdStringRevenue", new ArrayCompressedBigDecimal(new BigDecimal(expected))) + ); + + } +} + diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalCachingTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalCachingTest.java new file mode 100644 index 00000000000..70718403f5d --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalCachingTest.java @@ -0,0 +1,72 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Base64; +import java.util.HashMap; +import java.util.Map; + +public class CompressedBigDecimalCachingTest +{ + private static final Object FLAG = new Object(); + + @Test + public void testCrossFactory() + { + String name = "name"; + String fieldName = "fieldName"; + int size = 10; + int scale = 3; + boolean strictNumberParsing = true; + AggregatorFactory aggregatorFactory1 = new CompressedBigDecimalMaxAggregatorFactory( + name, + fieldName, + size, + scale, + strictNumberParsing + ); + AggregatorFactory aggregatorFactory2 = new CompressedBigDecimalMinAggregatorFactory( + name, + fieldName, + size, + scale, + strictNumberParsing + ); + AggregatorFactory aggregatorFactory3 = new CompressedBigDecimalSumAggregatorFactory( + name, + fieldName, + size, + scale, + strictNumberParsing + ); + + Map cache = new HashMap<>(); + + cache.put(Base64.getEncoder().encodeToString(aggregatorFactory1.getCacheKey()), FLAG); + cache.put(Base64.getEncoder().encodeToString(aggregatorFactory2.getCacheKey()), FLAG); + cache.put(Base64.getEncoder().encodeToString(aggregatorFactory3.getCacheKey()), FLAG); + + Assert.assertEquals(3, cache.size()); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalColumnTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalColumnTest.java new file mode 100644 index 00000000000..b779c631802 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalColumnTest.java @@ -0,0 +1,50 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.ColumnarMultiInts; +import org.apache.druid.segment.data.ReadableOffset; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class CompressedBigDecimalColumnTest +{ + @Test + public void testCompressedBigDecimalColumn() + { + ColumnarMultiInts columnarMultiInts = EasyMock.createMock(ColumnarMultiInts.class); + ColumnarInts columnarInts = EasyMock.createMock(ColumnarInts.class); + ReadableOffset readableOffset = EasyMock.createMock(ReadableOffset.class); + CompressedBigDecimalColumn compressedBigDecimalColumn = new CompressedBigDecimalColumn( + columnarInts, + columnarMultiInts + ); + Assert.assertEquals( + CompressedBigDecimalModule.COMPRESSED_BIG_DECIMAL, + compressedBigDecimalColumn.getTypeName() + ); + Assert.assertEquals(0, compressedBigDecimalColumn.getLength()); + Assert.assertEquals(CompressedBigDecimalColumn.class, compressedBigDecimalColumn.getClazz()); + Assert.assertNotNull(compressedBigDecimalColumn.makeColumnValueSelector(readableOffset)); + } + +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java new file mode 100644 index 00000000000..3562e6a35d6 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalFactoryTestBase.java @@ -0,0 +1,321 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.aggregation.AggregateCombiner; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.selector.TestColumnValueSelector; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Base64; + +/** + * common tests for {@link AggregatorFactory} implementations + */ +public abstract class CompressedBigDecimalFactoryTestBase +{ + private static final Object FLAG = new Object(); + + @Test + public abstract void testJsonSerialize() throws IOException; + + @Test + public abstract void testFinalizeComputation(); + + @Test + public abstract void testCompressedBigDecimalAggregatorFactoryDeserialize(); + + @Test + public abstract void testCombinerReset(); + + @Test + public abstract void testCombinerFold(); + + @Test + public abstract void testCompressedBigDecimalAggregateCombinerGetObject(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalAggregateCombinerGetLong(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalAggregateCombinerGetFloat(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalAggregateCombinerGetDouble(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalAggregatorGetFloat(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalAggregatorGetLong(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalBufferAggregatorGetFloat(); + + @Test(expected = UnsupportedOperationException.class) + public abstract void testCompressedBigDecimalBufferAggregatorGetLong(); + + @Test + public abstract void testCacheKeyEquality(); + + protected void testJsonSerializeHelper(Class clazz, T aggregatorFactory) throws IOException + { + ObjectMapper objectMapper = new ObjectMapper(); + + objectMapper.disable( + MapperFeature.AUTO_DETECT_CREATORS, + MapperFeature.AUTO_DETECT_FIELDS, + MapperFeature.AUTO_DETECT_GETTERS, + MapperFeature.AUTO_DETECT_IS_GETTERS + ); + + String jsonString = objectMapper.writeValueAsString(aggregatorFactory); + T deserializedAggregatorFactory = objectMapper.readValue(jsonString, clazz); + + Assert.assertEquals(aggregatorFactory, deserializedAggregatorFactory); + } + + @SuppressWarnings("ConstantConditions") + protected void testFinalizeComputationHelper(AggregatorFactory aggregatorFactory) + { + ArrayCompressedBigDecimal result1 = (ArrayCompressedBigDecimal) aggregatorFactory.finalizeComputation( + new ArrayCompressedBigDecimal(new BigDecimal("100.3141592")) + ); + + Assert.assertEquals("100.3141592", result1.toString()); + + ArrayCompressedBigDecimal result2 = (ArrayCompressedBigDecimal) aggregatorFactory.finalizeComputation( + new ArrayCompressedBigDecimal(new BigDecimal("0.000000000")) + ); + + Assert.assertEquals("0", result2.toString()); + + Object result3 = aggregatorFactory.finalizeComputation(null); + + Assert.assertNull(result3); + + ArrayCompressedBigDecimal result4 = (ArrayCompressedBigDecimal) aggregatorFactory.finalizeComputation( + new ArrayCompressedBigDecimal(new BigDecimal("1.000000000")) + ); + + Assert.assertEquals("1.000000000", result4.toString()); + + } + + protected void testCompressedBigDecimalAggregatorFactoryDeserializeHelper(AggregatorFactory aggregatorFactory) + { + CompressedBigDecimal compressedBigDecimal = (CompressedBigDecimal) aggregatorFactory.deserialize(5); + Assert.assertEquals("5", compressedBigDecimal.toString()); + } + + protected void testCompressedBigDecimalBufferAggregatorGetFloatHelper(BufferAggregator aggregator) + { + ByteBuffer byteBuffer = ByteBuffer.allocate(10); + aggregator.getFloat(byteBuffer, 0); + } + + protected void testCompressedBigDecimalBufferAggregatorGetLongHelper(BufferAggregator aggregator) + { + ByteBuffer byteBuffer = ByteBuffer.allocate(10); + aggregator.getLong(byteBuffer, 0); + } + + protected void testCombinerResetHelper(AggregateCombiner combiner) + { + TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( + CompressedBigDecimal.class, + ImmutableList.of(ArrayCompressedBigDecimal.wrap(new int[]{67, 0}, 0)), + DateTimes.of("2020-01-01") + ); + + columnValueSelector.advance(); + combiner.reset(columnValueSelector); + Assert.assertEquals("67", combiner.getObject().toString()); + } + + protected void testCombinerFoldHelper(AggregateCombiner combiner, String result1, String result2) + { + TestColumnValueSelector columnValueSelector = TestColumnValueSelector.of( + CompressedBigDecimal.class, + ImmutableList.of( + ArrayCompressedBigDecimal.wrap(new int[]{1, 0}, 0), + ArrayCompressedBigDecimal.wrap(new int[]{10, 0}, 0) + ), + DateTimes.of("2020-01-01") + ); + + columnValueSelector.advance(); + combiner.fold(columnValueSelector); + Assert.assertEquals(result1, combiner.getObject().toString()); + columnValueSelector.advance(); + combiner.fold(columnValueSelector); + Assert.assertEquals(result2, combiner.getObject().toString()); + } + + protected void testCompressedBigDecimalAggregateCombinerGetObjectHelper(AggregateCombiner combiner) + { + T compressedBigDecimal = combiner.getObject(); + Assert.assertSame(null, compressedBigDecimal); + } + + protected void testCompressedBigDecimalAggregateCombinerGetLongHelper(AggregateCombiner combiner) + { + combiner.getLong(); + } + + protected void testCompressedBigDecimalAggregateCombinerGetFloatHelper(AggregateCombiner combiner) + { + combiner.getFloat(); + } + + protected void testCompressedBigDecimalAggregateCombinerGetDoubleHelper(AggregateCombiner combiner) + { + combiner.getDouble(); + } + + protected void testCompressedBigDecimalAggregatorGetFloatHelper(Aggregator aggregator) + { + aggregator.getFloat(); + } + + protected void testCompressedBigDecimalAggregatorGetLongHelper(Aggregator aggregator) + { + aggregator.getLong(); + } + + /** + * creates a series of pairs of instances that should not be unique and verifies. Also tests when cache keys should + * be the same + */ + protected void testCacheKeyEqualityHelper(CompressedBigDecimalAggregatorFactoryCreator factoryCreator) + { + Assert.assertEquals( + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ), + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name2", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ) + ); + Assert.assertNotEquals( + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ), + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName2", + 10, + 3, + true + ).getCacheKey() + ) + ); + Assert.assertNotEquals( + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ), + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 6, + 3, + true + ).getCacheKey() + ) + ); + Assert.assertNotEquals( + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ), + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 9, + true + ).getCacheKey() + ) + ); + Assert.assertNotEquals( + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + true + ).getCacheKey() + ), + Base64.getEncoder().encodeToString( + factoryCreator.create( + "name1", + "fieldName1", + 10, + 3, + false + ).getCacheKey() + ) + ); + + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalGroupByQueryConfig.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalGroupByQueryConfig.java new file mode 100644 index 00000000000..df5e0d2101b --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalGroupByQueryConfig.java @@ -0,0 +1,69 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalGroupByQueryConfig +{ + private final String jsonQueryFile; + private final String jsonAggregatorsFile; + private final String stringRevenue; + private final String longRevenue; + private final String doubleRevenue; + + public CompressedBigDecimalGroupByQueryConfig( + String jsonQueryFile, + String jsonAggregatorsFile, + String stringRevenue, + String longRevenue, + String doubleRevenue + ) + { + this.jsonQueryFile = jsonQueryFile; + this.jsonAggregatorsFile = jsonAggregatorsFile; + this.stringRevenue = stringRevenue; + this.longRevenue = longRevenue; + this.doubleRevenue = doubleRevenue; + } + + public String getJsonQueryFile() + { + return jsonQueryFile; + } + + public String getJsonAggregatorsFile() + { + return jsonAggregatorsFile; + } + + public String getStringRevenue() + { + return stringRevenue; + } + + public String getLongRevenue() + { + return longRevenue; + } + + public String getDoubleRevenue() + { + return doubleRevenue; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorGroupByTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorGroupByTest.java new file mode 100644 index 00000000000..55e2ca6ec92 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorGroupByTest.java @@ -0,0 +1,62 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + + +public class CompressedBigDecimalMaxAggregatorGroupByTest extends CompressedBigDecimalAggregatorGroupByTestBase +{ + public CompressedBigDecimalMaxAggregatorGroupByTest( + GroupByQueryConfig config, + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig + ) + { + super(config, cbdGroupByQueryConfig); + } + + /** + * Constructor feeder. + * + * @return constructors + */ + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + List constructors = new ArrayList<>(); + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig = new CompressedBigDecimalGroupByQueryConfig( + "bd_max_test_groupby_query.json", + "bd_max_test_aggregators.json", + "9999999999.000000000", + "9999999999.000000000", + "9999999999.000000000" + ); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + constructors.add(new Object[]{config, cbdGroupByQueryConfig}); + } + return constructors; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTest.java new file mode 100644 index 00000000000..63e7f8558ef --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTest.java @@ -0,0 +1,38 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; + +public class CompressedBigDecimalMaxAggregatorTest extends CompressedBigDecimalAggregatorTestBase +{ + @Override + public void testNoData() + { + CompressedBigDecimalMaxAggregator aggregator = new CompressedBigDecimalMaxAggregator( + 4, + 9, + EasyMock.createMock(ColumnValueSelector.class), + false + ); + testNoDataHelper(aggregator, null); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTimeseriesTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTimeseriesTest.java new file mode 100644 index 00000000000..14e5ec61f23 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxAggregatorTimeseriesTest.java @@ -0,0 +1,44 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalMaxAggregatorTimeseriesTest extends CompressedBigDecimalAggregatorTimeseriesTestBase +{ + @Override + public void testIngestAndTimeseriesQuery() throws Exception + { + testIngestAndTimeseriesQueryHelper( + "bd_max_test_aggregators.json", + "bd_max_test_timeseries_query.json", + "9999999999.000000000" + ); + } + + @Override + public void testIngestMultipleSegmentsAndTimeseriesQuery() throws Exception + { + testIngestMultipleSegmentsAndTimeseriesQueryHelper( + "bd_max_test_aggregators.json", + "bd_max_test_timeseries_query.json", + "9999999999.000000000" + ); + } +} + diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java new file mode 100644 index 00000000000..0f1d548cec5 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxFactoryTest.java @@ -0,0 +1,216 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Arrays; + + +public class CompressedBigDecimalMaxFactoryTest extends CompressedBigDecimalFactoryTestBase +{ + @Test + public void testCompressedBigDecimalMaxAggregatorFactory() + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + Assert.assertEquals( + "CompressedBigDecimalMaxAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}", + aggregatorFactory.toString() + ); + Assert.assertNotNull(aggregatorFactory.getCacheKey()); + Assert.assertNull(aggregatorFactory.deserialize(null)); + Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); + Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); + Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); + Assert.assertEquals( + "[CompressedBigDecimalMaxAggregatorFactory{name='fieldName', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", + Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) + ); + Assert.assertNull(aggregatorFactory.combine(null, null)); + Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); + Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString()); + Assert.assertEquals( + "-4", + aggregatorFactory.combine( + new ArrayCompressedBigDecimal(new BigDecimal(-4)), + new ArrayCompressedBigDecimal(new BigDecimal(-10)) + ).toString() + ); + } + + @Override + @Test + public void testJsonSerialize() throws IOException + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + true + ); + + testJsonSerializeHelper(CompressedBigDecimalMaxAggregatorFactory.class, aggregatorFactory); + } + + @Override + @Test + public void testFinalizeComputation() + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testFinalizeComputationHelper(aggregatorFactory); + } + + @Override + @Test + public void testCompressedBigDecimalAggregatorFactoryDeserialize() + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testCompressedBigDecimalAggregatorFactoryDeserializeHelper(aggregatorFactory); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetFloat() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMaxBufferAggregator aggregator = new CompressedBigDecimalMaxBufferAggregator( + 4, + 0, + valueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMaxBufferAggregator aggregator = new CompressedBigDecimalMaxBufferAggregator( + 4, + 0, + valueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetLongHelper(aggregator); + } + + @Override + @Test + public void testCombinerReset() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCombinerResetHelper(combiner); + } + + @Override + @Test + public void testCombinerFold() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCombinerFoldHelper(combiner, "1", "10"); + } + + @Override + @Test + public void testCompressedBigDecimalAggregateCombinerGetObject() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetObjectHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetLong() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetLongHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetFloat() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetFloatHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetDouble() + { + CompressedBigDecimalMaxAggregateCombiner combiner = new CompressedBigDecimalMaxAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetDoubleHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregatorGetFloat() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMaxAggregator aggregator = new CompressedBigDecimalMaxAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMaxAggregator aggregator = new CompressedBigDecimalMaxAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetLongHelper(aggregator); + } + + @Override + public void testCacheKeyEquality() + { + testCacheKeyEqualityHelper(CompressedBigDecimalMaxAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java new file mode 100644 index 00000000000..fec14699d40 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMaxSqlAggregatorTest.java @@ -0,0 +1,84 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.sql.calcite.planner.DruidOperatorTable; + +public class CompressedBigDecimalMaxSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase +{ + private static final String FUNCTION_NAME = CompressedBigDecimalMaxSqlAggregator.NAME; + + @Override + public DruidOperatorTable createOperatorTable() + { + return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalMaxSqlAggregator()), ImmutableSet.of()); + } + + @Override + public void testCompressedBigDecimalAggWithNumberParse() + { + + testCompressedBigDecimalAggWithNumberParseHelper( + FUNCTION_NAME, + new Object[]{"6.000000000", "6.000000000", "10.100000000"}, + CompressedBigDecimalMaxAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggWithStrictNumberParse() + { + testCompressedBigDecimalAggWithStrictNumberParseHelper( + FUNCTION_NAME, + CompressedBigDecimalMaxAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScale() + { + testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"6.000", "6.000", "10.100"}, + CompressedBigDecimalMaxAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultScale() + { + testCompressedBigDecimalAggDefaultScaleHelper( + FUNCTION_NAME, + new Object[]{"6.000000000", "6.000000000", "10.100000000"}, + CompressedBigDecimalMaxAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultSizeAndScale() + { + testCompressedBigDecimalAggDefaultSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"6.000000000", "6.000000000", "10.100000000"}, + CompressedBigDecimalMaxAggregatorFactory::new + ); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorGroupByTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorGroupByTest.java new file mode 100644 index 00000000000..1567da9ed02 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorGroupByTest.java @@ -0,0 +1,64 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + + +@RunWith(Parameterized.class) +public class CompressedBigDecimalMinAggregatorGroupByTest extends CompressedBigDecimalAggregatorGroupByTestBase +{ + public CompressedBigDecimalMinAggregatorGroupByTest( + GroupByQueryConfig config, + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig + ) + { + super(config, cbdGroupByQueryConfig); + } + + /** + * Constructor feeder. + * + * @return constructors + */ + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + List constructors = new ArrayList<>(); + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig = new CompressedBigDecimalGroupByQueryConfig( + "bd_min_test_groupby_query.json", + "bd_min_test_aggregators.json", + "-1.000000000", + "-1.000000000", + "-1.000000000" + ); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + constructors.add(new Object[]{config, cbdGroupByQueryConfig}); + } + return constructors; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTest.java new file mode 100644 index 00000000000..9f6bf646846 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTest.java @@ -0,0 +1,38 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; + +public class CompressedBigDecimalMinAggregatorTest extends CompressedBigDecimalAggregatorTestBase +{ + @Override + public void testNoData() + { + CompressedBigDecimalMinAggregator aggregator = new CompressedBigDecimalMinAggregator( + 4, + 9, + EasyMock.createMock(ColumnValueSelector.class), + false + ); + testNoDataHelper(aggregator, null); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTimeseriesTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTimeseriesTest.java new file mode 100644 index 00000000000..a22c3605af8 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinAggregatorTimeseriesTest.java @@ -0,0 +1,44 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalMinAggregatorTimeseriesTest extends CompressedBigDecimalAggregatorTimeseriesTestBase +{ + @Override + public void testIngestAndTimeseriesQuery() throws Exception + { + testIngestAndTimeseriesQueryHelper( + "bd_min_test_aggregators.json", + "bd_min_test_timeseries_query.json", + "-1.000000000" + ); + } + + @Override + public void testIngestMultipleSegmentsAndTimeseriesQuery() throws Exception + { + testIngestMultipleSegmentsAndTimeseriesQueryHelper( + "bd_min_test_aggregators.json", + "bd_min_test_timeseries_query.json", + "-1.000000000" + ); + } +} + diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java new file mode 100644 index 00000000000..1c445011ff6 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinFactoryTest.java @@ -0,0 +1,217 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Arrays; + + +public class CompressedBigDecimalMinFactoryTest extends CompressedBigDecimalFactoryTestBase +{ + @Test + public void testCompressedBigDecimalMinAggregatorFactory() + { + CompressedBigDecimalMinAggregatorFactory aggregatorFactory = new CompressedBigDecimalMinAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + Assert.assertEquals( + "CompressedBigDecimalMinAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}", + aggregatorFactory.toString() + ); + Assert.assertNotNull(aggregatorFactory.getCacheKey()); + Assert.assertNull(aggregatorFactory.deserialize(null)); + Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); + Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); + Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); + Assert.assertEquals( + "[CompressedBigDecimalMinAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", + Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) + ); + // default is to initialize to + Assert.assertNull(aggregatorFactory.combine(null, null)); + Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); + Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString()); + Assert.assertEquals( + "4", + aggregatorFactory.combine( + new ArrayCompressedBigDecimal(new BigDecimal(4)), + new ArrayCompressedBigDecimal(new BigDecimal(10)) + ).toString() + ); + } + + @Override + @Test + public void testJsonSerialize() throws IOException + { + CompressedBigDecimalMinAggregatorFactory aggregatorFactory = new CompressedBigDecimalMinAggregatorFactory( + "name", + "fieldName", + 9, + 0, + true + ); + + testJsonSerializeHelper(CompressedBigDecimalMinAggregatorFactory.class, aggregatorFactory); + } + + @Override + @Test + public void testFinalizeComputation() + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testFinalizeComputationHelper(aggregatorFactory); + } + + @Override + @Test + public void testCompressedBigDecimalAggregatorFactoryDeserialize() + { + CompressedBigDecimalMinAggregatorFactory aggregatorFactory = new CompressedBigDecimalMinAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testCompressedBigDecimalAggregatorFactoryDeserializeHelper(aggregatorFactory); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetFloat() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMinBufferAggregator aggregator = new CompressedBigDecimalMinBufferAggregator( + 4, + 0, + valueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMinBufferAggregator aggregator = new CompressedBigDecimalMinBufferAggregator( + 4, + 0, + valueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetLongHelper(aggregator); + } + + @Override + @Test + public void testCombinerReset() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCombinerResetHelper(combiner); + } + + @Override + @Test + public void testCombinerFold() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCombinerFoldHelper(combiner, "1", "1"); + } + + @Override + @Test + public void testCompressedBigDecimalAggregateCombinerGetObject() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetObjectHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetLong() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetLongHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetFloat() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetFloatHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetDouble() + { + CompressedBigDecimalMinAggregateCombiner combiner = new CompressedBigDecimalMinAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetDoubleHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregatorGetFloat() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMinAggregator aggregator = new CompressedBigDecimalMinAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalMinAggregator aggregator = new CompressedBigDecimalMinAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetLongHelper(aggregator); + } + + @Override + public void testCacheKeyEquality() + { + testCacheKeyEqualityHelper(CompressedBigDecimalMinAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java new file mode 100644 index 00000000000..0031780ee30 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalMinSqlAggregatorTest.java @@ -0,0 +1,83 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.sql.calcite.planner.DruidOperatorTable; + +public class CompressedBigDecimalMinSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase +{ + private static final String FUNCTION_NAME = CompressedBigDecimalMinSqlAggregator.NAME; + + @Override + public DruidOperatorTable createOperatorTable() + { + return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalMinSqlAggregator()), ImmutableSet.of()); + } + + @Override + public void testCompressedBigDecimalAggWithNumberParse() + { + testCompressedBigDecimalAggWithNumberParseHelper( + FUNCTION_NAME, + new Object[]{"1.000000000", "1.000000000", "0"}, + CompressedBigDecimalMinAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggWithStrictNumberParse() + { + testCompressedBigDecimalAggWithStrictNumberParseHelper( + FUNCTION_NAME, + CompressedBigDecimalMinAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScale() + { + testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"1.000", "1.000", "0"}, + CompressedBigDecimalMinAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultScale() + { + testCompressedBigDecimalAggDefaultScaleHelper( + FUNCTION_NAME, + new Object[]{"1.000000000", "1.000000000", "0"}, + CompressedBigDecimalMinAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultSizeAndScale() + { + testCompressedBigDecimalAggDefaultSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"1.000000000", "1.000000000", "0"}, + CompressedBigDecimalMinAggregatorFactory::new + ); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java similarity index 68% rename from extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTest.java rename to extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java index 7eb6b1febb5..d433bbe372d 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTest.java +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSqlAggregatorTestBase.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -32,6 +31,7 @@ import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.MapInputRowParser; import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -57,7 +57,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -public class CompressedBigDecimalSqlAggregatorTest extends BaseCalciteQueryTest +public abstract class CompressedBigDecimalSqlAggregatorTestBase extends BaseCalciteQueryTest { private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( @@ -117,12 +117,6 @@ public class CompressedBigDecimalSqlAggregatorTest extends BaseCalciteQueryTest return walker; } - @Override - public DruidOperatorTable createOperatorTable() - { - return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalSqlAggregator()), ImmutableSet.of()); - } - @Override public ObjectMapper createQueryJsonMapper() { @@ -132,132 +126,162 @@ public class CompressedBigDecimalSqlAggregatorTest extends BaseCalciteQueryTest return objectMapper; } + @Override + public abstract DruidOperatorTable createOperatorTable(); + @Test - public void testCompressedBigDecimalAggWithNumberParse1() - { - cannotVectorize(); - testQuery( - "SELECT big_sum(m1, 9, 9), big_sum(m2, 9, 9), big_sum(dim1, 9, 9, false) FROM foo", - Collections.singletonList( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .granularity(Granularities.ALL) - .aggregators( - new CompressedBigDecimalAggregatorFactory("a0:agg", "m1", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a1:agg", "m2", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a2:agg", "dim1", 9, 9, false) - ) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of(new Object[]{ - "21.000000000", - "21.000000000", - "13.100000000", - }) - ); - } + public abstract void testCompressedBigDecimalAggWithNumberParse(); @Test(expected = NumberFormatException.class) - public void testCompressedBigDecimalAggWithNumberParse2() + public abstract void testCompressedBigDecimalAggWithStrictNumberParse(); + + @Test + public abstract void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScale(); + + @Test + public abstract void testCompressedBigDecimalAggDefaultScale(); + + @Test + public abstract void testCompressedBigDecimalAggDefaultSizeAndScale(); + + protected void testCompressedBigDecimalAggWithNumberParseHelper( + String functionName, + Object[] expectedResults, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) { cannotVectorize(); testQuery( - "SELECT big_sum(dim1, 9, 9, true) FROM foo", + StringUtils.format( + "SELECT %s(m1, 9, 9), %s(m2, 9, 9), %s(dim1, 9, 9, false) FROM foo", + functionName, + functionName, + functionName + ), Collections.singletonList( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) .aggregators( - new CompressedBigDecimalAggregatorFactory("a0:agg", "dim1", 9, 9, true) + factoryCreator.create("a0:agg", "m1", 9, 9, false), + factoryCreator.create("a1:agg", "m2", 9, 9, false), + factoryCreator.create("a2:agg", "dim1", 9, 9, false) + ) .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of(new Object[]{"13.100000000"}) + ImmutableList.of(expectedResults) ); } - @Test - public void testCompressedBigDecimalAggDefaultNumberParse() + protected void testCompressedBigDecimalAggWithStrictNumberParseHelper( + String functionName, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) { cannotVectorize(); testQuery( - "SELECT big_sum(m1, 9, 9), big_sum(m2, 9, 9), big_sum(dim1, 9, 9) FROM foo", + StringUtils.format("SELECT %s(dim1, 9, 9, true) FROM foo", functionName), Collections.singletonList( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .aggregators( - new CompressedBigDecimalAggregatorFactory("a0:agg", "m1", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a1:agg", "m2", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a2:agg", "dim1", 9, 9, false) - ) + .aggregators(factoryCreator.create("a0:agg", "dim1", 9, 9, true)) .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of(new Object[]{ - "21.000000000", - "21.000000000", - "13.100000000", - }) + ImmutableList.of(new Object[]{"unused"}) ); } - @Test - public void testCompressedBigDecimalAggDefaultScale() + public void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScaleHelper( + String functionName, + Object[] expectedResults, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) { cannotVectorize(); testQuery( - "SELECT big_sum(m1, 9), big_sum(m2, 9), big_sum(dim1, 9) FROM foo", + StringUtils.format( + "SELECT %s(m1, 9, 3), %s(m2, 9, 3), %s(dim1, 9, 3) FROM foo", + functionName, + functionName, + functionName + ), Collections.singletonList( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) .aggregators( - new CompressedBigDecimalAggregatorFactory("a0:agg", "m1", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a1:agg", "m2", 9, 9, false), - new CompressedBigDecimalAggregatorFactory("a2:agg", "dim1", 9, 9, false) + factoryCreator.create("a0:agg", "m1", 9, 3, false), + factoryCreator.create("a1:agg", "m2", 9, 3, false), + factoryCreator.create("a2:agg", "dim1", 9, 3, false) ) .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of(new Object[]{ - "21.000000000", - "21.000000000", - "13.100000000" - }) + ImmutableList.of(expectedResults) ); } - @Test - public void testCompressedBigDecimalAggDefaultSizeAndScale() + public void testCompressedBigDecimalAggDefaultScaleHelper( + String functionName, + Object[] expectedResults, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) { cannotVectorize(); testQuery( - "SELECT big_sum(m1), big_sum(m2), big_sum(dim1) FROM foo", + StringUtils.format( + "SELECT %s(m1, 9), %s(m2, 9), %s(dim1, 9) FROM foo", + functionName, + functionName, + functionName + ), Collections.singletonList( Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) .aggregators( - new CompressedBigDecimalAggregatorFactory("a0:agg", "m1", 6, 9, false), - new CompressedBigDecimalAggregatorFactory("a1:agg", "m2", 6, 9, false), - new CompressedBigDecimalAggregatorFactory("a2:agg", "dim1", 6, 9, false) + factoryCreator.create("a0:agg", "m1", 9, 9, false), + factoryCreator.create("a1:agg", "m2", 9, 9, false), + factoryCreator.create("a2:agg", "dim1", 9, 9, false) ) .context(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of(new Object[]{ - "21.000000000", - "21.000000000", - "13.100000000" - }) + ImmutableList.of(expectedResults) + ); + } + + public void testCompressedBigDecimalAggDefaultSizeAndScaleHelper( + String functionName, + Object[] expectedResults, + CompressedBigDecimalAggregatorFactoryCreator factoryCreator + ) + { + cannotVectorize(); + testQuery( + StringUtils.format("SELECT %s(m1), %s(m2), %s(dim1) FROM foo", functionName, functionName, functionName), + Collections.singletonList( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) + .granularity(Granularities.ALL) + .aggregators( + factoryCreator.create("a0:agg", "m1", 6, 9, false), + factoryCreator.create("a1:agg", "m2", 6, 9, false), + factoryCreator.create("a2:agg", "dim1", 6, 9, false) + ) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of(expectedResults) ); } diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorGroupByTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorGroupByTest.java new file mode 100644 index 00000000000..f308665723e --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorGroupByTest.java @@ -0,0 +1,62 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + + +public class CompressedBigDecimalSumAggregatorGroupByTest extends CompressedBigDecimalAggregatorGroupByTestBase +{ + public CompressedBigDecimalSumAggregatorGroupByTest( + GroupByQueryConfig config, + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig + ) + { + super(config, cbdGroupByQueryConfig); + } + + /** + * Constructor feeder. + * + * @return constructors + */ + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() + { + List constructors = new ArrayList<>(); + CompressedBigDecimalGroupByQueryConfig cbdGroupByQueryConfig = new CompressedBigDecimalGroupByQueryConfig( + "bd_sum_test_groupby_query.json", + "bd_sum_test_aggregators.json", + "15000000010.000000005", + "10000000010.000000000", + "15000000010.500000000" + ); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + constructors.add(new Object[]{config, cbdGroupByQueryConfig}); + } + return constructors; + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTest.java new file mode 100644 index 00000000000..dfdf1a348b8 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTest.java @@ -0,0 +1,40 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; + +import java.math.BigDecimal; + +public class CompressedBigDecimalSumAggregatorTest extends CompressedBigDecimalAggregatorTestBase +{ + @Override + public void testNoData() + { + CompressedBigDecimalSumAggregator aggregator = new CompressedBigDecimalSumAggregator( + 4, + 9, + EasyMock.createMock(ColumnValueSelector.class), + false + ); + testNoDataHelper(aggregator, new ArrayCompressedBigDecimal(BigDecimal.ZERO)); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTimeseriesTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTimeseriesTest.java new file mode 100644 index 00000000000..8c67ccc35ed --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumAggregatorTimeseriesTest.java @@ -0,0 +1,44 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +public class CompressedBigDecimalSumAggregatorTimeseriesTest extends CompressedBigDecimalAggregatorTimeseriesTestBase +{ + @Override + public void testIngestAndTimeseriesQuery() throws Exception + { + testIngestAndTimeseriesQueryHelper( + "bd_sum_test_aggregators.json", + "bd_sum_test_timeseries_query.json", + "15000000010.000000005" + ); + } + + @Override + public void testIngestMultipleSegmentsAndTimeseriesQuery() throws Exception + { + testIngestMultipleSegmentsAndTimeseriesQueryHelper( + "bd_sum_test_aggregators.json", + "bd_sum_test_timeseries_query.json", + "15000000010.000000005" + ); + } +} + diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java new file mode 100644 index 00000000000..c26a2568f7f --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumFactoryTest.java @@ -0,0 +1,212 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Arrays; + +/** + * test CompressedBigDecimalSumFactory and various aggregators and combiner produced + */ +public class CompressedBigDecimalSumFactoryTest extends CompressedBigDecimalFactoryTestBase +{ + @Test + public void testCompressedBigDecimalAggregatorFactory() + { + CompressedBigDecimalSumAggregatorFactory aggregatorFactory = new CompressedBigDecimalSumAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + Assert.assertEquals( + "CompressedBigDecimalSumAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}", + aggregatorFactory.toString() + ); + Assert.assertNotNull(aggregatorFactory.getCacheKey()); + Assert.assertNull(aggregatorFactory.deserialize(null)); + Assert.assertEquals("5", aggregatorFactory.deserialize(new BigDecimal(5)).toString()); + Assert.assertEquals("5.0", aggregatorFactory.deserialize(5d).toString()); + Assert.assertEquals("5", aggregatorFactory.deserialize("5").toString()); + Assert.assertEquals( + "[CompressedBigDecimalSumAggregatorFactory{name='name', type='COMPLEX', fieldName='fieldName', requiredFields='[fieldName]', size='9', scale='0', strictNumberParsing='false'}]", + Arrays.toString(aggregatorFactory.getRequiredColumns().toArray()) + ); + Assert.assertEquals("0", aggregatorFactory.combine(null, null).toString()); + Assert.assertEquals("4", aggregatorFactory.combine(new BigDecimal(4), null).toString()); + Assert.assertEquals("4", aggregatorFactory.combine(null, new BigDecimal(4)).toString()); + Assert.assertEquals( + "8", + aggregatorFactory.combine( + new ArrayCompressedBigDecimal(new BigDecimal(4)), + new ArrayCompressedBigDecimal(new BigDecimal(4)) + ).toString() + ); + } + + @Override + public void testJsonSerialize() throws IOException + { + CompressedBigDecimalSumAggregatorFactory aggregatorFactory = new CompressedBigDecimalSumAggregatorFactory( + "name", + "fieldName", + 9, + 0, + true + ); + + testJsonSerializeHelper(CompressedBigDecimalSumAggregatorFactory.class, aggregatorFactory); + } + + @Override + public void testFinalizeComputation() + { + CompressedBigDecimalMaxAggregatorFactory aggregatorFactory = new CompressedBigDecimalMaxAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testFinalizeComputationHelper(aggregatorFactory); + } + + @Override + public void testCompressedBigDecimalAggregatorFactoryDeserialize() + { + CompressedBigDecimalSumAggregatorFactory aggregatorFactory = new CompressedBigDecimalSumAggregatorFactory( + "name", + "fieldName", + 9, + 0, + false + ); + + testCompressedBigDecimalAggregatorFactoryDeserializeHelper(aggregatorFactory); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetFloat() + { + ColumnValueSelector columnValueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalSumBufferAggregator aggregator = new CompressedBigDecimalSumBufferAggregator( + 4, + 0, + columnValueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalBufferAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalSumBufferAggregator aggregator = new CompressedBigDecimalSumBufferAggregator( + 4, + 0, + valueSelector, + false + ); + + testCompressedBigDecimalBufferAggregatorGetLongHelper(aggregator); + } + + @Override + public void testCombinerReset() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCombinerResetHelper(combiner); + } + + @Override + public void testCombinerFold() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCombinerFoldHelper(combiner, "1", "11"); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetObject() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetObjectHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetLong() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetLongHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetFloat() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetFloatHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregateCombinerGetDouble() + { + CompressedBigDecimalSumAggregateCombiner combiner = new CompressedBigDecimalSumAggregateCombiner(); + + testCompressedBigDecimalAggregateCombinerGetDoubleHelper(combiner); + } + + @Override + public void testCompressedBigDecimalAggregatorGetFloat() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalSumAggregator aggregator = new CompressedBigDecimalSumAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetFloatHelper(aggregator); + } + + @Override + public void testCompressedBigDecimalAggregatorGetLong() + { + ColumnValueSelector valueSelector = EasyMock.createMock(ColumnValueSelector.class); + CompressedBigDecimalSumAggregator aggregator = new CompressedBigDecimalSumAggregator(2, 0, valueSelector, false); + + testCompressedBigDecimalAggregatorGetLongHelper(aggregator); + } + + @Override + public void testCacheKeyEquality() + { + testCacheKeyEqualityHelper(CompressedBigDecimalSumAggregatorFactory::new); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java new file mode 100644 index 00000000000..c7bc86d1543 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/java/org/apache/druid/compressedbigdecimal/CompressedBigDecimalSumSqlAggregatorTest.java @@ -0,0 +1,84 @@ +/* + * 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. 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 + * + * 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 org.apache.druid.compressedbigdecimal; + +import com.google.common.collect.ImmutableSet; +import org.apache.druid.sql.calcite.planner.DruidOperatorTable; + +public class CompressedBigDecimalSumSqlAggregatorTest extends CompressedBigDecimalSqlAggregatorTestBase +{ + private static final String FUNCTION_NAME = CompressedBigDecimalSumSqlAggregator.NAME; + + @Override + public DruidOperatorTable createOperatorTable() + { + return new DruidOperatorTable(ImmutableSet.of(new CompressedBigDecimalSumSqlAggregator()), ImmutableSet.of()); + } + + @Override + public void testCompressedBigDecimalAggWithNumberParse() + { + testCompressedBigDecimalAggWithNumberParseHelper( + FUNCTION_NAME, + new Object[]{"21.000000000", "21.000000000", "13.100000000"}, + CompressedBigDecimalSumAggregatorFactory::new + + ); + } + + @Override + public void testCompressedBigDecimalAggWithStrictNumberParse() + { + testCompressedBigDecimalAggWithStrictNumberParseHelper( + FUNCTION_NAME, + CompressedBigDecimalSumAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScale() + { + testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"21.000", "21.000", "13.100"}, + CompressedBigDecimalSumAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultScale() + { + testCompressedBigDecimalAggDefaultScaleHelper( + FUNCTION_NAME, + new Object[]{"21.000000000", "21.000000000", "13.100000000"}, + CompressedBigDecimalSumAggregatorFactory::new + ); + } + + @Override + public void testCompressedBigDecimalAggDefaultSizeAndScale() + { + testCompressedBigDecimalAggDefaultSizeAndScaleHelper( + FUNCTION_NAME, + new Object[]{"21.000000000", "21.000000000", "13.100000000"}, + CompressedBigDecimalSumAggregatorFactory::new + ); + } +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_aggregators.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_aggregators.json similarity index 74% rename from extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_aggregators.json rename to extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_aggregators.json index 9db44909684..b97dc7f0b5f 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_aggregators.json +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_aggregators.json @@ -1,6 +1,6 @@ [ { - "type": "compressedBigDecimal", + "type": "compressedBigDecimalMax", "name": "bigDecimalRevenue", "fieldName": "revenue", "scale": 9, diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_groupby_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_groupby_query.json similarity index 82% rename from extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_groupby_query.json rename to extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_groupby_query.json index 471167e8dd7..a28938866ee 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_groupby_query.json +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_groupby_query.json @@ -6,21 +6,21 @@ ], "aggregations": [ { - "type": "compressedBigDecimal", + "type": "compressedBigDecimalMax", "name": "cbdRevenueFromString", "fieldName": "revenue", "scale": 9, "size": 3 }, { - "type": "compressedBigDecimal", + "type": "compressedBigDecimalMax", "name": "cbdRevenueFromLong", "fieldName": "longRevenue", "scale": 9, "size": 3 }, { - "type": "compressedBigDecimal", + "type": "compressedBigDecimalMax", "name": "cbdRevenueFromDouble", "fieldName": "doubleRevenue", "scale": 9, diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_timeseries_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_timeseries_query.json similarity index 84% rename from extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_timeseries_query.json rename to extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_timeseries_query.json index 92a1f4457b8..85e563dac5a 100644 --- a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_test_timeseries_query.json +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_max_test_timeseries_query.json @@ -4,8 +4,8 @@ "granularity": "ALL", "aggregations": [ { - "type": "compressedBigDecimal", - "name": "revenue", + "type": "compressedBigDecimalMax", + "name": "cbdStringRevenue", "fieldName": "revenue", "scale": 9, "size": 3 diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_aggregators.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_aggregators.json new file mode 100644 index 00000000000..7618418b234 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_aggregators.json @@ -0,0 +1,9 @@ +[ + { + "type": "compressedBigDecimalMin", + "name": "bigDecimalRevenue", + "fieldName": "revenue", + "scale": 9, + "size": 3 + } +] diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_groupby_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_groupby_query.json new file mode 100644 index 00000000000..284c4050e53 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_groupby_query.json @@ -0,0 +1,33 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [ + ], + "aggregations": [ + { + "type": "compressedBigDecimalMin", + "name": "cbdRevenueFromString", + "fieldName": "revenue", + "scale": 9, + "size": 3 + }, + { + "type": "compressedBigDecimalMin", + "name": "cbdRevenueFromLong", + "fieldName": "longRevenue", + "scale": 9, + "size": 3 + }, + { + "type": "compressedBigDecimalMin", + "name": "cbdRevenueFromDouble", + "fieldName": "doubleRevenue", + "scale": 9, + "size": 3 + } + ], + "intervals": [ + "2017-01-01T00:00:00.000Z/P1D" + ] +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_timeseries_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_timeseries_query.json new file mode 100644 index 00000000000..5ceebe030af --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_min_test_timeseries_query.json @@ -0,0 +1,25 @@ +{ + "queryType": "timeseries", + "dataSource": "test_datasource", + "granularity": "ALL", + "aggregations": [ + { + "type": "compressedBigDecimalMin", + "name": "cbdStringRevenue", + "fieldName": "revenue", + "scale": 9, + "size": 3 + } + ], + "filter": { + "type": "not", + "field": { + "type": "selector", + "dimension": "property", + "value": "XXX" + } + }, + "intervals": [ + "2017-01-01T00:00:00.000Z/P1D" + ] +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_aggregators.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_aggregators.json new file mode 100644 index 00000000000..53b12eb83d6 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_aggregators.json @@ -0,0 +1,9 @@ +[ + { + "type": "compressedBigDecimalSum", + "name": "bigDecimalRevenue", + "fieldName": "revenue", + "scale": 9, + "size": 3 + } +] diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_groupby_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_groupby_query.json new file mode 100644 index 00000000000..b580c19f439 --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_groupby_query.json @@ -0,0 +1,33 @@ +{ + "queryType": "groupBy", + "dataSource": "test_datasource", + "granularity": "ALL", + "dimensions": [ + ], + "aggregations": [ + { + "type": "compressedBigDecimalSum", + "name": "cbdRevenueFromString", + "fieldName": "revenue", + "scale": 9, + "size": 3 + }, + { + "type": "compressedBigDecimalSum", + "name": "cbdRevenueFromLong", + "fieldName": "longRevenue", + "scale": 9, + "size": 3 + }, + { + "type": "compressedBigDecimalSum", + "name": "cbdRevenueFromDouble", + "fieldName": "doubleRevenue", + "scale": 9, + "size": 3 + } + ], + "intervals": [ + "2017-01-01T00:00:00.000Z/P1D" + ] +} diff --git a/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_timeseries_query.json b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_timeseries_query.json new file mode 100644 index 00000000000..44a4880f85b --- /dev/null +++ b/extensions-contrib/compressed-bigdecimal/src/test/resources/bd_sum_test_timeseries_query.json @@ -0,0 +1,25 @@ +{ + "queryType": "timeseries", + "dataSource": "test_datasource", + "granularity": "ALL", + "aggregations": [ + { + "type": "compressedBigDecimalSum", + "name": "cbdStringRevenue", + "fieldName": "revenue", + "scale": 9, + "size": 3 + } + ], + "filter": { + "type": "not", + "field": { + "type": "selector", + "dimension": "property", + "value": "XXX" + } + }, + "intervals": [ + "2017-01-01T00:00:00.000Z/P1D" + ] +}