CompressedBigDecimal Min/Max (#13141)

This adds min/max functions for CompressedBigDecimal. It exposes these
functions via sql (BIG_MAX, BIG_MIN--see the SqlAggFunction
implementations).

It also includes various bug fixes and cleanup to the original
CompressedBigDecimal code include the AggregatorFactories. Various null
handling was improved.

Additional test cases were added for both new and existing code
including a base test case for AggregationFactories. Other tests common
across sum,min,max may be refactored also to share the varoius cases in
the future.
This commit is contained in:
Sam Rash 2022-10-11 16:35:21 -07:00 committed by GitHub
parent 9688674ea8
commit 80e10ffe22
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
64 changed files with 4433 additions and 1113 deletions

View File

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

View File

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

View File

@ -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<CompressedBigDecimal>
{
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 <b>this</b>
*/
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
* <p>
* 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());
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;
}
}
}

View File

@ -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<CompressedBigDecimal>
{
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<CompressedBigDecimal> classOfObject()
{
return CompressedBigDecimal.class;
}
}

View File

@ -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<CompressedBigDecimal> selector;
protected final boolean strictNumberParsing;
protected final CompressedBigDecimal value;
protected boolean empty;
private final ColumnValueSelector<CompressedBigDecimal> 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<CompressedBigDecimal> 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()
{

View File

@ -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<ColumnValueSelector<CompressedBigDecimal>>
{
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<CompressedBigDecimal> 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<CompressedBigDecimal> selector(ColumnSelectorFactory metricFactory)
{
return (ColumnValueSelector<CompressedBigDecimal>) metricFactory.makeColumnValueSelector(fieldName);
}
@Override
protected Aggregator factorize(
ColumnSelectorFactory metricFactory,
@Nonnull ColumnValueSelector<CompressedBigDecimal> selector
)
{
return new CompressedBigDecimalAggregator(size, scale, selector, strictNumberParsing);
}
@Override
protected BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
@Nonnull ColumnValueSelector<CompressedBigDecimal> selector
)
{
return new CompressedBigDecimalBufferAggregator(size, scale, selector, strictNumberParsing);
}
/* (non-Javadoc)
* @see org.apache.druid.query.aggregation.AggregatorFactory#getComparator()
*/
@Override
public Comparator<CompressedBigDecimal> 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<CompressedBigDecimal> makeAggregateCombiner()
{
return new CompressedBigDecimalAggregateCombiner();
}
/* (non-Javadoc)
* @see org.apache.druid.query.aggregation.AggregatorFactory#getRequiredColumns()
*/
@Override
public List<AggregatorFactory> 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<String> 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() + '\'' +
'}';
}
}

View File

@ -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<ColumnValueSelector<CompressedBigDecimal>>
{
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<CompressedBigDecimal> 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<CompressedBigDecimal> selector(ColumnSelectorFactory metricFactory)
{
return (ColumnValueSelector<CompressedBigDecimal>) metricFactory.makeColumnValueSelector(fieldName);
}
@Override
protected abstract Aggregator factorize(
ColumnSelectorFactory metricFactory,
ColumnValueSelector<CompressedBigDecimal> selector
);
@Override
protected abstract BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
ColumnValueSelector<CompressedBigDecimal> selector
);
@Override
public Comparator<CompressedBigDecimal> getComparator()
{
return COMPARATOR;
}
@Nullable
@Override
public abstract Object combine(Object lhs, Object rhs);
@Override
public abstract AggregatorFactory getCombiningFactory();
@Override
public abstract AggregateCombiner<CompressedBigDecimal> makeAggregateCombiner();
@Override
public abstract List<AggregatorFactory> getRequiredColumns();
@Override
public abstract String toString();
@Nullable
@Override
public Object deserialize(Object object)
{
return Utils.objToCompressedBigDecimal(object);
}
@Override
public List<String> 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);
}
}

View File

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

View File

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

View File

@ -31,10 +31,6 @@ import java.io.IOException;
public class CompressedBigDecimalJsonSerializer extends JsonSerializer<CompressedBigDecimal>
{
/* (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

View File

@ -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<CompressedBigDecimal> selector =
(ColumnValueSelector<CompressedBigDecimal>) 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<CompressedBigDecimal> selector =
(ColumnValueSelector<CompressedBigDecimal>) columnValueSelector;
CompressedBigDecimal cbd = selector.getObject();
if (value == null) {
value = new ArrayCompressedBigDecimal(cbd);
} else {
value.accumulateMax(cbd);
}
}
}

View File

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

View File

@ -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<CompressedBigDecimal> selector
)
{
return new CompressedBigDecimalMaxAggregator(size, scale, selector, strictNumberParsing);
}
@Override
protected BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
ColumnValueSelector<CompressedBigDecimal> 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<CompressedBigDecimal> makeAggregateCombiner()
{
return new CompressedBigDecimalMaxAggregateCombiner();
}
@Override
public List<AggregatorFactory> 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() + '\'' +
'}';
}
}

View File

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

View File

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

View File

@ -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<CompressedBigDecimal> selector =
(ColumnValueSelector<CompressedBigDecimal>) 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<CompressedBigDecimal> selector =
(ColumnValueSelector<CompressedBigDecimal>) columnValueSelector;
CompressedBigDecimal cbd = selector.getObject();
if (value == null) {
value = new ArrayCompressedBigDecimal(cbd);
} else {
value.accumulateMin(cbd);
}
}
}

View File

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

View File

@ -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<CompressedBigDecimal> selector
)
{
return new CompressedBigDecimalMinAggregator(size, scale, selector, strictNumberParsing);
}
@Override
public BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
@Nonnull ColumnValueSelector<CompressedBigDecimal> 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<CompressedBigDecimal> makeAggregateCombiner()
{
return new CompressedBigDecimalMinAggregateCombiner();
}
@Override
public List<AggregatorFactory> 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() + '\'' +
'}';
}
}

View File

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

View File

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

View File

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

View File

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

View File

@ -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<CompressedBigDecimal>
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<CompressedBigDecimal>) 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<CompressedBigDecimal> classOfObject()
{
return CompressedBigDecimal.class;
}
}

View File

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

View File

@ -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<CompressedBigDecimal> selector
)
{
return new CompressedBigDecimalSumAggregator(size, scale, selector, strictNumberParsing);
}
@Override
protected BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
@Nonnull ColumnValueSelector<CompressedBigDecimal> 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<CompressedBigDecimal> makeAggregateCombiner()
{
return new CompressedBigDecimalSumAggregateCombiner();
}
@Override
public List<AggregatorFactory> 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() + '\'' +
'}';
}
}

View File

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

View File

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

View File

@ -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 <b>this</b>
*/
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 <b>this</b>
*/
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 <b>this</b>
*/
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<ByteBuffer, Integer>, ObjBiIntConsumer<ByteBuffer>
{

View File

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

View File

@ -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 d3 = ArrayCompressedBigDecimal.allocateZero(
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE,
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE
);
ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocate(
CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE,
CompressedBigDecimalAggregatorFactory.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,13 +188,13 @@ public class ArrayCompressedBigDecimalTest
@Test(expected = IAE.class)
public void testCompareToFailOptimized()
{
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 + 1
ArrayCompressedBigDecimal d2 = ArrayCompressedBigDecimal.allocateZero(
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE,
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE + 1
);
// size doesn't match, so an IAE will be thrown
@ -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 d2 = ArrayCompressedBigDecimal.allocateZero(
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 2,
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE
);
ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocate(
CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 3,
CompressedBigDecimalAggregatorFactory.DEFAULT_SCALE
ArrayCompressedBigDecimal d3 = ArrayCompressedBigDecimal.allocateZero(
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SIZE + 3,
CompressedBigDecimalAggregatorFactoryBase.DEFAULT_SCALE
);
ArrayCompressedBigDecimal d4 = ArrayCompressedBigDecimal.allocate(
CompressedBigDecimalAggregatorFactory.DEFAULT_SIZE + 4,
CompressedBigDecimalAggregatorFactory.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());
}
}

View File

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

View File

@ -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<Object[]> 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()))
)
);
}
}

View File

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

View File

@ -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<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getValue();
Map<String, Object> event = result.getBaseObject();
assertEquals(new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))),
((Result<TimeseriesResultValue>) 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<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getValue();
Map<String, Object> event = result.getBaseObject();
assertEquals(new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))),
((Result<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getTimestamp());
assertThat(event, aMapWithSize(1));
assertThat(event, hasEntry("revenue", new BigDecimal("15000000010.000000005")));
}
}

View File

@ -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<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getValue();
Map<String, Object> event = result.getBaseObject();
assertEquals(
new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))),
((Result<TimeseriesResultValue>) 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<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getValue();
Map<String, Object> event = result.getBaseObject();
assertEquals(
new DateTime("2017-01-01T00:00:00Z", DateTimeZone.forTimeZone(TimeZone.getTimeZone("UTC"))),
((Result<TimeseriesResultValue>) Iterables.getOnlyElement(seq.toList())).getTimestamp()
);
assertThat(event, aMapWithSize(1));
assertThat(
event,
hasEntry("cbdStringRevenue", new ArrayCompressedBigDecimal(new BigDecimal(expected)))
);
}
}

View File

@ -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<String, Object> 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());
}
}

View File

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

View File

@ -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 <T> void testJsonSerializeHelper(Class<T> 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 <T> void testCombinerResetHelper(AggregateCombiner<T> combiner)
{
TestColumnValueSelector<CompressedBigDecimal> 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 <T> void testCombinerFoldHelper(AggregateCombiner<T> combiner, String result1, String result2)
{
TestColumnValueSelector<CompressedBigDecimal> 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 <T> void testCompressedBigDecimalAggregateCombinerGetObjectHelper(AggregateCombiner<T> combiner)
{
T compressedBigDecimal = combiner.getObject();
Assert.assertSame(null, compressedBigDecimal);
}
protected <T> void testCompressedBigDecimalAggregateCombinerGetLongHelper(AggregateCombiner<T> combiner)
{
combiner.getLong();
}
protected <T> void testCompressedBigDecimalAggregateCombinerGetFloatHelper(AggregateCombiner<T> combiner)
{
combiner.getFloat();
}
protected <T> void testCompressedBigDecimalAggregateCombinerGetDoubleHelper(AggregateCombiner<T> 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()
)
);
}
}

View File

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

View File

@ -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<Object[]> 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;
}
}

View File

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

View File

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

View File

@ -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<compressedBigDecimal>', 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<compressedBigDecimal>', 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<CompressedBigDecimal> valueSelector = EasyMock.createMock(ColumnValueSelector.class);
CompressedBigDecimalMaxBufferAggregator aggregator = new CompressedBigDecimalMaxBufferAggregator(
4,
0,
valueSelector,
false
);
testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator);
}
@Override
public void testCompressedBigDecimalBufferAggregatorGetLong()
{
ColumnValueSelector<CompressedBigDecimal> 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);
}
}

View File

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

View File

@ -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<Object[]> 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;
}
}

View File

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

View File

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

View File

@ -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<compressedBigDecimal>', 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<compressedBigDecimal>', 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<CompressedBigDecimal> valueSelector = EasyMock.createMock(ColumnValueSelector.class);
CompressedBigDecimalMinBufferAggregator aggregator = new CompressedBigDecimalMinBufferAggregator(
4,
0,
valueSelector,
false
);
testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator);
}
@Override
public void testCompressedBigDecimalBufferAggregatorGetLong()
{
ColumnValueSelector<CompressedBigDecimal> 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);
}
}

View File

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

View File

@ -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<Map<String, Object>> 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()
{
cannotVectorize();
testQuery(
"SELECT big_sum(dim1, 9, 9, true) FROM foo",
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)
)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(new Object[]{"13.100000000"})
);
}
public abstract void testCompressedBigDecimalAggWithStrictNumberParse();
@Test
public void testCompressedBigDecimalAggDefaultNumberParse()
{
cannotVectorize();
testQuery(
"SELECT big_sum(m1, 9, 9), big_sum(m2, 9, 9), big_sum(dim1, 9, 9) 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 testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScale();
@Test
public void testCompressedBigDecimalAggDefaultScale()
{
cannotVectorize();
testQuery(
"SELECT big_sum(m1, 9), big_sum(m2, 9), big_sum(dim1, 9) 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 testCompressedBigDecimalAggDefaultScale();
@Test
public void testCompressedBigDecimalAggDefaultSizeAndScale()
public abstract void testCompressedBigDecimalAggDefaultSizeAndScale();
protected void testCompressedBigDecimalAggWithNumberParseHelper(
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, 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", "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)
);
}
protected void testCompressedBigDecimalAggWithStrictNumberParseHelper(
String functionName,
CompressedBigDecimalAggregatorFactoryCreator factoryCreator
)
{
cannotVectorize();
testQuery(
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(factoryCreator.create("a0:agg", "dim1", 9, 9, true))
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(new Object[]{"unused"})
);
}
public void testCompressedBigDecimalAggDefaultNumberParseAndCustomSizeAndScaleHelper(
String functionName,
Object[] expectedResults,
CompressedBigDecimalAggregatorFactoryCreator factoryCreator
)
{
cannotVectorize();
testQuery(
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(
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(expectedResults)
);
}
public void testCompressedBigDecimalAggDefaultScaleHelper(
String functionName,
Object[] expectedResults,
CompressedBigDecimalAggregatorFactoryCreator factoryCreator
)
{
cannotVectorize();
testQuery(
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(
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(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)
);
}

View File

@ -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<Object[]> 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;
}
}

View File

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

View File

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

View File

@ -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<compressedBigDecimal>', 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<compressedBigDecimal>', 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<CompressedBigDecimal> columnValueSelector = EasyMock.createMock(ColumnValueSelector.class);
CompressedBigDecimalSumBufferAggregator aggregator = new CompressedBigDecimalSumBufferAggregator(
4,
0,
columnValueSelector,
false
);
testCompressedBigDecimalBufferAggregatorGetFloatHelper(aggregator);
}
@Override
public void testCompressedBigDecimalBufferAggregatorGetLong()
{
ColumnValueSelector<CompressedBigDecimal> 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);
}
}

View File

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

View File

@ -1,6 +1,6 @@
[
{
"type": "compressedBigDecimal",
"type": "compressedBigDecimalMax",
"name": "bigDecimalRevenue",
"fieldName": "revenue",
"scale": 9,

View File

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

View File

@ -4,8 +4,8 @@
"granularity": "ALL",
"aggregations": [
{
"type": "compressedBigDecimal",
"name": "revenue",
"type": "compressedBigDecimalMax",
"name": "cbdStringRevenue",
"fieldName": "revenue",
"scale": 9,
"size": 3

View File

@ -0,0 +1,9 @@
[
{
"type": "compressedBigDecimalMin",
"name": "bigDecimalRevenue",
"fieldName": "revenue",
"scale": 9,
"size": 3
}
]

View File

@ -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"
]
}

View File

@ -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"
]
}

View File

@ -0,0 +1,9 @@
[
{
"type": "compressedBigDecimalSum",
"name": "bigDecimalRevenue",
"fieldName": "revenue",
"scale": 9,
"size": 3
}
]

View File

@ -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"
]
}

View File

@ -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"
]
}