first/last aggregators and nulls (#9161)

* null handling for numeric first/last aggregators, refactor to not extend nullable numeric agg since they are complex typed aggs

* initially null or not based on config

* review stuff, make string first/last consistent with null handling of numeric columns, more tests

* docs

* handle nil selectors, revert to primitive first/last types so groupby v1 works...
This commit is contained in:
Clint Wylie 2020-01-20 11:51:54 -08:00 committed by Jonathan Wei
parent 180c622e0f
commit 8011211a0c
40 changed files with 1619 additions and 593 deletions

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Pair;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.Comparator;
public class SerializablePair<T1, T2> extends Pair<T1, T2> public class SerializablePair<T1, T2> extends Pair<T1, T2>
{ {
@ -45,4 +46,25 @@ public class SerializablePair<T1, T2> extends Pair<T1, T2>
{ {
return rhs; return rhs;
} }
public static <T1, T2> Comparator<SerializablePair<T1, T2>> createNullHandlingComparator(
Comparator<T2> delegate,
boolean nullsFirst
)
{
final int firstIsNull = nullsFirst ? -1 : 1;
final int secondIsNull = nullsFirst ? 1 : -1;
return (o1, o2) -> {
if (o1 == null || o1.rhs == null) {
if (o2 == null || o2.rhs == null) {
return 0;
}
return firstIsNull;
}
if (o2 == null || o2.rhs == null) {
return secondIsNull;
}
return delegate.compare(o1.rhs, o2.rhs);
};
}
} }

View File

@ -136,7 +136,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `doubleFirst` aggregator #### `doubleFirst` aggregator
`doubleFirst` computes the metric value with the minimum timestamp or 0 if no row exist `doubleFirst` computes the metric value with the minimum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {
@ -148,7 +148,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `doubleLast` aggregator #### `doubleLast` aggregator
`doubleLast` computes the metric value with the maximum timestamp or 0 if no row exist `doubleLast` computes the metric value with the maximum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {
@ -160,7 +160,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `floatFirst` aggregator #### `floatFirst` aggregator
`floatFirst` computes the metric value with the minimum timestamp or 0 if no row exist `floatFirst` computes the metric value with the minimum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {
@ -172,7 +172,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `floatLast` aggregator #### `floatLast` aggregator
`floatLast` computes the metric value with the maximum timestamp or 0 if no row exist `floatLast` computes the metric value with the maximum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {
@ -184,7 +184,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `longFirst` aggregator #### `longFirst` aggregator
`longFirst` computes the metric value with the minimum timestamp or 0 if no row exist `longFirst` computes the metric value with the minimum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {
@ -196,7 +196,7 @@ Note that queries with first/last aggregators on a segment created with rollup e
#### `longLast` aggregator #### `longLast` aggregator
`longLast` computes the metric value with the maximum timestamp or 0 if no row exist `longLast` computes the metric value with the maximum timestamp or 0 in default mode or `null` in SQL compatible mode if no row exist
```json ```json
{ {

View File

@ -199,9 +199,9 @@ Only the COUNT aggregation can accept DISTINCT.
|`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.| |`STDDEV_POP(expr)`|Computes standard deviation population of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.| |`STDDEV_SAMP(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.| |`STDDEV(expr)`|Computes standard deviation sample of `expr`. See [stats extension](../development/extensions-core/stats.html) documentation for additional details.|
|`EARLIEST(expr)`|Returns the earliest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "earliest" is the value first encountered with the minimum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.| |`EARLIEST(expr)`|Returns the earliest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "earliest" is the value first encountered with the minimum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the first value encountered.|
|`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.| |`EARLIEST(expr, maxBytesPerString)`|Like `EARLIEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
|`LATEST(expr)`|Returns the latest non-null value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.| |`LATEST(expr)`|Returns the latest value of `expr`, which must be numeric. If `expr` comes from a relation with a timestamp column (like a Druid datasource) then "latest" is the value last encountered with the maximum overall timestamp of all values being aggregated. If `expr` does not come from a relation with a timestamp, then it is simply the last value encountered.|
|`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.| |`LATEST(expr, maxBytesPerString)`|Like `LATEST(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|
|`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can return the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will return any non-null value of `expr`| |`ANY_VALUE(expr)`|Returns any value of `expr`, which must be numeric. If `druid.generic.useDefaultValueForNull=true` this can return the default value for null and does not prefer "non-null" values over the default value for null. If `druid.generic.useDefaultValueForNull=false`, then this will return any non-null value of `expr`|
|`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.| |`ANY_VALUE(expr, maxBytesPerString)`|Like `ANY_VALUE(expr)`, but for strings. The `maxBytesPerString` parameter determines how much aggregation space to allocate per string. Strings longer than this limit will be truncated. This parameter should be set as low as possible, since high values will lead to wasted memory.|

View File

@ -20,42 +20,29 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class DoubleFirstAggregator implements Aggregator public class DoubleFirstAggregator extends NumericFirstAggregator<BaseDoubleColumnValueSelector>
{ {
double firstValue;
private final BaseDoubleColumnValueSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
protected long firstTime;
protected double firstValue;
public DoubleFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector) public DoubleFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector)
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
firstTime = Long.MAX_VALUE;
firstValue = 0; firstValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); firstValue = valueSelector.getDouble();
if (time < firstTime) {
firstTime = time;
firstValue = valueSelector.getDouble();
}
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(firstTime, firstValue); return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
} }
@Override @Override
@ -75,11 +62,5 @@ public class DoubleFirstAggregator implements Aggregator
{ {
return (long) firstValue; return (long) firstValue;
} }
@Override
public void close()
{
}
} }

View File

@ -30,10 +30,11 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -45,10 +46,34 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class DoubleFirstAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new DoubleFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new DoubleFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
public static final Comparator<SerializablePair<Long, Double>> VALUE_COMPARATOR = public static final Comparator<SerializablePair<Long, Double>> VALUE_COMPARATOR =
Comparator.comparingDouble(o -> o.rhs); SerializablePair.createNullHandlingComparator(Double::compare, true);
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -69,24 +94,31 @@ public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFacto
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new DoubleFirstAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new DoubleFirstAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new DoubleFirstBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new DoubleFirstBufferAggregator( valueSelector
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), );
selector }
);
} }
@Override @Override
@ -126,35 +158,54 @@ public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFacto
return new DoubleFirstAggregatorFactory(name, name) return new DoubleFirstAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleFirstAggregator(null, null) return new DoubleFirstAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject(); SerializablePair<Long, Double> pair = selector.getObject();
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
firstTime = pair.lhs; firstTime = pair.lhs;
firstValue = pair.rhs; if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleFirstBufferAggregator(null, null) return new DoubleFirstBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Double> pair = selector.getObject();
buf.putDouble(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject(); SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
long firstTime = buf.getLong(position); long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putDouble(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -178,6 +229,9 @@ public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFacto
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
} }
@ -221,16 +275,15 @@ public class DoubleFirstAggregatorFactory extends NullableNumericAggregatorFacto
@Override @Override
public String getTypeName() public String getTypeName()
{ {
if (storeDoubleAsFloat) { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return "float"; return storeDoubleAsFloat ? "float" : "double";
}
return "double";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES + Double.BYTES; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Double.BYTES;
} }
@Override @Override

View File

@ -20,79 +20,55 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class DoubleFirstBufferAggregator implements BufferAggregator public class DoubleFirstBufferAggregator extends NumericFirstBufferAggregator<BaseDoubleColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector;
private final BaseDoubleColumnValueSelector valueSelector;
public DoubleFirstBufferAggregator( public DoubleFirstBufferAggregator(
BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector timeSelector,
BaseDoubleColumnValueSelector valueSelector BaseDoubleColumnValueSelector valueSelector
) )
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MAX_VALUE); buf.putDouble(position, 0);
buf.putDouble(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putDouble(position, valueSelector.getDouble());
long firstTime = buf.getLong(position);
if (time < firstTime) {
buf.putLong(position, time);
buf.putDouble(position + Long.BYTES, valueSelector.getDouble());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getDouble(position + Long.BYTES)); final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return (float) buf.getDouble(position + Long.BYTES); return (float) buf.getDouble(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return (long) buf.getDouble(position + Long.BYTES); return (long) buf.getDouble(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return buf.getDouble(position + Long.BYTES); return buf.getDouble(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

View File

@ -20,45 +20,32 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class FloatFirstAggregator implements Aggregator public class FloatFirstAggregator extends NumericFirstAggregator<BaseFloatColumnValueSelector>
{ {
float firstValue;
private final BaseFloatColumnValueSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
protected long firstTime;
protected float firstValue;
public FloatFirstAggregator( public FloatFirstAggregator(
BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector BaseFloatColumnValueSelector valueSelector
) )
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
firstTime = Long.MAX_VALUE;
firstValue = 0; firstValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); firstValue = valueSelector.getFloat();
if (time < firstTime) {
firstTime = time;
firstValue = valueSelector.getFloat();
}
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(firstTime, firstValue); return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
} }
@Override @Override
@ -70,7 +57,7 @@ public class FloatFirstAggregator implements Aggregator
@Override @Override
public double getDouble() public double getDouble()
{ {
return (double) firstValue; return firstValue;
} }
@Override @Override
@ -78,11 +65,5 @@ public class FloatFirstAggregator implements Aggregator
{ {
return (long) firstValue; return (long) firstValue;
} }
@Override
public void close()
{
}
} }

View File

@ -30,10 +30,11 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -45,10 +46,34 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public class FloatFirstAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class FloatFirstAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new FloatFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new FloatFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
public static final Comparator<SerializablePair<Long, Float>> VALUE_COMPARATOR = public static final Comparator<SerializablePair<Long, Float>> VALUE_COMPARATOR =
Comparator.comparingDouble(o -> o.rhs); SerializablePair.createNullHandlingComparator(Float::compare, true);
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -67,24 +92,31 @@ public class FloatFirstAggregatorFactory extends NullableNumericAggregatorFactor
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new FloatFirstAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new FloatFirstAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new FloatFirstBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new FloatFirstBufferAggregator( valueSelector
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), );
selector }
);
} }
@Override @Override
@ -121,38 +153,56 @@ public class FloatFirstAggregatorFactory extends NullableNumericAggregatorFactor
@Override @Override
public AggregatorFactory getCombiningFactory() public AggregatorFactory getCombiningFactory()
{ {
return new FloatFirstAggregatorFactory(name, name) return new FloatFirstAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatFirstAggregator(null, null) return new FloatFirstAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject(); SerializablePair<Long, Float> pair = selector.getObject();
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
firstTime = pair.lhs; firstTime = pair.lhs;
firstValue = pair.rhs; if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatFirstBufferAggregator(null, null) return new FloatFirstBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Float> pair = selector.getObject();
buf.putFloat(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject(); SerializablePair<Long, Float> pair = selector.getObject();
long firstTime = buf.getLong(position); long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putFloat(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -176,6 +226,9 @@ public class FloatFirstAggregatorFactory extends NullableNumericAggregatorFactor
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
} }
@ -219,13 +272,15 @@ public class FloatFirstAggregatorFactory extends NullableNumericAggregatorFactor
@Override @Override
public String getTypeName() public String getTypeName()
{ {
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return "float"; return "float";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES + Float.BYTES; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Float.BYTES;
} }
@Override @Override

View File

@ -20,79 +20,55 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class FloatFirstBufferAggregator implements BufferAggregator public class FloatFirstBufferAggregator extends NumericFirstBufferAggregator<BaseFloatColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector;
private final BaseFloatColumnValueSelector valueSelector;
public FloatFirstBufferAggregator( public FloatFirstBufferAggregator(
BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector BaseFloatColumnValueSelector valueSelector
) )
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MAX_VALUE); buf.putFloat(position, 0);
buf.putFloat(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putFloat(position, valueSelector.getFloat());
long firstTime = buf.getLong(position);
if (time < firstTime) {
buf.putLong(position, time);
buf.putFloat(position + Long.BYTES, valueSelector.getFloat());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getFloat(position + Long.BYTES)); final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return buf.getFloat(position + Long.BYTES); return buf.getFloat(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return (long) buf.getFloat(position + Long.BYTES); return (long) buf.getFloat(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return (double) buf.getFloat(position + Long.BYTES); return buf.getFloat(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

View File

@ -20,41 +20,28 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class LongFirstAggregator implements Aggregator public class LongFirstAggregator extends NumericFirstAggregator<BaseLongColumnValueSelector>
{ {
long firstValue;
private final BaseLongColumnValueSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
protected long firstTime;
protected long firstValue;
public LongFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector) public LongFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
firstTime = Long.MAX_VALUE;
firstValue = 0; firstValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); firstValue = valueSelector.getLong();
if (time < firstTime) {
firstTime = time;
firstValue = valueSelector.getLong();
}
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(firstTime, firstValue); return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
} }
@Override @Override
@ -74,10 +61,4 @@ public class LongFirstAggregator implements Aggregator
{ {
return firstValue; return firstValue;
} }
@Override
public void close()
{
}
} }

View File

@ -30,10 +30,11 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -44,10 +45,34 @@ import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
public class LongFirstAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class LongFirstAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new LongFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new LongFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
public static final Comparator<SerializablePair<Long, Long>> VALUE_COMPARATOR = public static final Comparator<SerializablePair<Long, Long>> VALUE_COMPARATOR =
Comparator.comparingLong(o -> o.rhs); SerializablePair.createNullHandlingComparator(Long::compare, true);
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -66,24 +91,31 @@ public class LongFirstAggregatorFactory extends NullableNumericAggregatorFactory
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new LongFirstAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new LongFirstAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new LongFirstBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new LongFirstBufferAggregator( valueSelector
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), );
selector }
);
} }
@Override @Override
@ -123,35 +155,52 @@ public class LongFirstAggregatorFactory extends NullableNumericAggregatorFactory
return new LongFirstAggregatorFactory(name, name) return new LongFirstAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongFirstAggregator(null, null) return new LongFirstAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject(); SerializablePair<Long, Long> pair = selector.getObject();
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
firstTime = pair.lhs; firstTime = pair.lhs;
firstValue = pair.rhs; if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongFirstBufferAggregator(null, null) return new LongFirstBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Long> pair = selector.getObject();
buf.putLong(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject(); SerializablePair<Long, Long> pair = selector.getObject();
long firstTime = buf.getLong(position); long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) { if (pair.lhs < firstTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putLong(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -175,6 +224,9 @@ public class LongFirstAggregatorFactory extends NullableNumericAggregatorFactory
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
} }
@ -218,13 +270,15 @@ public class LongFirstAggregatorFactory extends NullableNumericAggregatorFactory
@Override @Override
public String getTypeName() public String getTypeName()
{ {
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return "long"; return "long";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES * 2; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Long.BYTES;
} }
@Override @Override

View File

@ -20,75 +20,51 @@
package org.apache.druid.query.aggregation.first; package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class LongFirstBufferAggregator implements BufferAggregator public class LongFirstBufferAggregator extends NumericFirstBufferAggregator<BaseLongColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector;
private final BaseLongColumnValueSelector valueSelector;
public LongFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector) public LongFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MAX_VALUE); buf.putLong(position, 0);
buf.putLong(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putLong(position, valueSelector.getLong());
long firstTime = buf.getLong(position);
if (time < firstTime) {
buf.putLong(position, time);
buf.putLong(position + Long.BYTES, valueSelector.getLong());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getLong(position + Long.BYTES)); final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return (float) buf.getLong(position + Long.BYTES); return (float) buf.getLong(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return buf.getLong(position + Long.BYTES); return buf.getLong(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return (double) buf.getLong(position + Long.BYTES); return (double) buf.getLong(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

View File

@ -0,0 +1,74 @@
/*
* 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.query.aggregation.first;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
/**
* Base type for on heap 'first' aggregator for primitive numeric column selectors
*/
public abstract class NumericFirstAggregator<TSelector extends BaseNullableColumnValueSelector> implements Aggregator
{
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
long firstTime;
boolean rhsNull;
public NumericFirstAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
firstTime = Long.MAX_VALUE;
rhsNull = !useDefault;
}
/**
* Store the current primitive typed 'first' value
*/
abstract void setCurrentValue();
@Override
public void aggregate()
{
long time = timeSelector.getLong();
if (time < firstTime) {
firstTime = time;
if (useDefault || !valueSelector.isNull()) {
setCurrentValue();
rhsNull = false;
} else {
rhsNull = true;
}
}
}
@Override
public void close()
{
// nothing to close
}
}

View File

@ -0,0 +1,112 @@
/*
* 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.query.aggregation.first;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
import java.nio.ByteBuffer;
/**
* Base type for buffer based 'first' aggregator for primitive numeric column selectors
*/
public abstract class NumericFirstBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
implements BufferAggregator
{
static final int NULL_OFFSET = Long.BYTES;
static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
public NumericFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
}
/**
* Initialize the buffer value at the position of {@link #VALUE_OFFSET}
*/
abstract void initValue(ByteBuffer buf, int position);
/**
* Place the primitive value in the buffer at the position of {@link #VALUE_OFFSET}
*/
abstract void putValue(ByteBuffer buf, int position);
void updateTimeWithValue(ByteBuffer buf, int position, long time)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET);
}
void updateTimeWithNull(ByteBuffer buf, int position, long time)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NULL_BYTE);
}
boolean isValueNull(ByteBuffer buf, int position)
{
return buf.get(position + NULL_OFFSET) == NullHandling.IS_NULL_BYTE;
}
@Override
public void init(ByteBuffer buf, int position)
{
buf.putLong(position, Long.MAX_VALUE);
buf.put(position + NULL_OFFSET, useDefault ? NullHandling.IS_NOT_NULL_BYTE : NullHandling.IS_NULL_BYTE);
initValue(buf, position + VALUE_OFFSET);
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
long time = timeSelector.getLong();
long firstTime = buf.getLong(position);
if (time < firstTime) {
if (useDefault || !valueSelector.isNull()) {
updateTimeWithValue(buf, position, time);
} else {
updateTimeWithNull(buf, position, time);
}
}
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
}
}

View File

@ -64,7 +64,7 @@ public class StringFirstAggregator implements Aggregator
valueSelector valueSelector
); );
if (inPair != null && inPair.rhs != null && inPair.lhs < firstTime) { if (inPair != null && inPair.lhs < firstTime) {
firstTime = inPair.lhs; firstTime = inPair.lhs;
firstValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes); firstValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes);
} }
@ -73,11 +73,8 @@ public class StringFirstAggregator implements Aggregator
if (time < firstTime) { if (time < firstTime) {
final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject()); final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject());
firstTime = time;
if (value != null) { firstValue = StringUtils.fastLooseChop(value, maxStringBytes);
firstTime = time;
firstValue = StringUtils.fastLooseChop(value, maxStringBytes);
}
} }
} }
} }

View File

@ -34,9 +34,11 @@ import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
@ -47,6 +49,34 @@ import java.util.Objects;
@JsonTypeName("stringFirst") @JsonTypeName("stringFirst")
public class StringFirstAggregatorFactory extends AggregatorFactory public class StringFirstAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new StringFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance(),
0,
false
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new StringFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance(),
0,
false
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
public static final int DEFAULT_MAX_STRING_SIZE = 1024; public static final int DEFAULT_MAX_STRING_SIZE = 1024;
public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare( public static final Comparator TIME_COMPARATOR = (o1, o2) -> Longs.compare(
@ -120,24 +150,32 @@ public class StringFirstAggregatorFactory extends AggregatorFactory
public Aggregator factorize(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName); final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName);
return new StringFirstAggregator( if (valueSelector instanceof NilColumnValueSelector) {
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), return NIL_AGGREGATOR;
valueSelector, } else {
maxStringBytes, return new StringFirstAggregator(
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName)) metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
); valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
);
}
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName); final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName);
return new StringFirstBufferAggregator( if (valueSelector instanceof NilColumnValueSelector) {
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), return NIL_BUFFER_AGGREGATOR;
valueSelector, } else {
maxStringBytes, return new StringFirstBufferAggregator(
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName)) metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
); valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
);
}
} }
@Override @Override

View File

@ -71,7 +71,7 @@ public class StringFirstBufferAggregator implements BufferAggregator
valueSelector valueSelector
); );
if (inPair != null && inPair.rhs != null) { if (inPair != null) {
final long firstTime = buf.getLong(position); final long firstTime = buf.getLong(position);
if (inPair.lhs < firstTime) { if (inPair.lhs < firstTime) {
StringFirstLastUtils.writePair( StringFirstLastUtils.writePair(
@ -89,14 +89,12 @@ public class StringFirstBufferAggregator implements BufferAggregator
if (time < firstTime) { if (time < firstTime) {
final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject()); final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject());
if (value != null) { StringFirstLastUtils.writePair(
StringFirstLastUtils.writePair( buf,
buf, position,
position, new SerializablePairLongString(time, value),
new SerializablePairLongString(time, value), maxStringBytes
maxStringBytes );
);
}
} }
} }
} }

View File

@ -20,42 +20,29 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class DoubleLastAggregator implements Aggregator public class DoubleLastAggregator extends NumericLastAggregator<BaseDoubleColumnValueSelector>
{ {
double lastValue;
private final BaseDoubleColumnValueSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
protected long lastTime;
protected double lastValue;
public DoubleLastAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector) public DoubleLastAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector)
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
lastTime = Long.MIN_VALUE;
lastValue = 0; lastValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); lastValue = valueSelector.getDouble();
if (time >= lastTime) {
lastTime = time;
lastValue = valueSelector.getDouble();
}
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(lastTime, lastValue); return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
} }
@Override @Override
@ -75,10 +62,4 @@ public class DoubleLastAggregator implements Aggregator
{ {
return lastValue; return lastValue;
} }
@Override
public void close()
{
}
} }

View File

@ -30,12 +30,13 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.aggregation.first.DoubleFirstAggregatorFactory; import org.apache.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -47,8 +48,31 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public class DoubleLastAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class DoubleLastAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new DoubleLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new DoubleLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -68,24 +92,31 @@ public class DoubleLastAggregatorFactory extends NullableNumericAggregatorFactor
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new DoubleLastAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new DoubleLastAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new DoubleLastBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new DoubleLastBufferAggregator( valueSelector
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), );
selector }
);
} }
@Override @Override
@ -125,35 +156,54 @@ public class DoubleLastAggregatorFactory extends NullableNumericAggregatorFactor
return new DoubleLastAggregatorFactory(name, name) return new DoubleLastAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleLastAggregator(null, null) return new DoubleLastAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject(); SerializablePair<Long, Double> pair = selector.getObject();
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
lastTime = pair.lhs; lastTime = pair.lhs;
lastValue = pair.rhs; if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleLastBufferAggregator(null, null) return new DoubleLastBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Double> pair = selector.getObject();
buf.putDouble(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject(); SerializablePair<Long, Double> pair = selector.getObject();
long lastTime = buf.getLong(position); long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putDouble(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -177,6 +227,9 @@ public class DoubleLastAggregatorFactory extends NullableNumericAggregatorFactor
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
} }
@ -220,17 +273,15 @@ public class DoubleLastAggregatorFactory extends NullableNumericAggregatorFactor
@Override @Override
public String getTypeName() public String getTypeName()
{ {
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
if (storeDoubleAsFloat) { return storeDoubleAsFloat ? "float" : "double";
return "float";
}
return "double";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES + Double.BYTES; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Double.BYTES;
} }
@Override @Override

View File

@ -20,79 +20,55 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class DoubleLastBufferAggregator implements BufferAggregator public class DoubleLastBufferAggregator extends NumericLastBufferAggregator<BaseDoubleColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector;
private final BaseDoubleColumnValueSelector valueSelector;
public DoubleLastBufferAggregator( public DoubleLastBufferAggregator(
BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector timeSelector,
BaseDoubleColumnValueSelector valueSelector BaseDoubleColumnValueSelector valueSelector
) )
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MIN_VALUE); buf.putDouble(position, 0);
buf.putDouble(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putDouble(position, valueSelector.getDouble());
long lastTime = buf.getLong(position);
if (time >= lastTime) {
buf.putLong(position, time);
buf.putDouble(position + Long.BYTES, valueSelector.getDouble());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getDouble(position + Long.BYTES)); final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return (float) buf.getDouble(position + Long.BYTES); return (float) buf.getDouble(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return (long) buf.getDouble(position + Long.BYTES); return (long) buf.getDouble(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return buf.getDouble(position + Long.BYTES); return buf.getDouble(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

View File

@ -20,42 +20,29 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class FloatLastAggregator implements Aggregator public class FloatLastAggregator extends NumericLastAggregator<BaseFloatColumnValueSelector>
{ {
float lastValue;
private final BaseFloatColumnValueSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
protected long lastTime;
protected float lastValue;
public FloatLastAggregator(BaseLongColumnValueSelector timeSelector, BaseFloatColumnValueSelector valueSelector) public FloatLastAggregator(BaseLongColumnValueSelector timeSelector, BaseFloatColumnValueSelector valueSelector)
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
lastTime = Long.MIN_VALUE;
lastValue = 0; lastValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); lastValue = valueSelector.getFloat();
if (time >= lastTime) {
lastTime = time;
lastValue = valueSelector.getFloat();
}
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(lastTime, lastValue); return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
} }
@Override @Override
@ -73,12 +60,6 @@ public class FloatLastAggregator implements Aggregator
@Override @Override
public double getDouble() public double getDouble()
{ {
return (double) lastValue; return lastValue;
}
@Override
public void close()
{
} }
} }

View File

@ -30,12 +30,13 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.aggregation.first.FloatFirstAggregatorFactory; import org.apache.druid.query.aggregation.first.FloatFirstAggregatorFactory;
import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -47,8 +48,31 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public class FloatLastAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class FloatLastAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new FloatLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new FloatLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -66,24 +90,31 @@ public class FloatLastAggregatorFactory extends NullableNumericAggregatorFactory
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new FloatLastAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new FloatLastAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new FloatLastBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new FloatLastBufferAggregator( valueSelector
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), );
selector }
);
} }
@Override @Override
@ -123,35 +154,52 @@ public class FloatLastAggregatorFactory extends NullableNumericAggregatorFactory
return new FloatLastAggregatorFactory(name, name) return new FloatLastAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatLastAggregator(null, null) return new FloatLastAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject(); SerializablePair<Long, Float> pair = selector.getObject();
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
lastTime = pair.lhs; lastTime = pair.lhs;
lastValue = pair.rhs; if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatLastBufferAggregator(null, null) return new FloatLastBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Float> pair = selector.getObject();
buf.putFloat(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Float> pair = (SerializablePair<Long, Float>) selector.getObject(); SerializablePair<Long, Float> pair = selector.getObject();
long lastTime = buf.getLong(position); long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putFloat(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -175,6 +223,9 @@ public class FloatLastAggregatorFactory extends NullableNumericAggregatorFactory
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
} }
@ -219,13 +270,15 @@ public class FloatLastAggregatorFactory extends NullableNumericAggregatorFactory
@Override @Override
public String getTypeName() public String getTypeName()
{ {
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return "float"; return "float";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES + Float.BYTES; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Float.BYTES;
} }
@Override @Override

View File

@ -20,76 +20,55 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class FloatLastBufferAggregator implements BufferAggregator public class FloatLastBufferAggregator extends NumericLastBufferAggregator<BaseFloatColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector; public FloatLastBufferAggregator(
private final BaseFloatColumnValueSelector valueSelector; BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector
public FloatLastBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseFloatColumnValueSelector valueSelector) )
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MIN_VALUE); buf.putFloat(position, 0);
buf.putFloat(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putFloat(position, valueSelector.getFloat());
long lastTime = buf.getLong(position);
if (time >= lastTime) {
buf.putLong(position, time);
buf.putFloat(position + Long.BYTES, valueSelector.getFloat());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getFloat(position + Long.BYTES)); final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return buf.getFloat(position + Long.BYTES); return buf.getFloat(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return (long) buf.getFloat(position + Long.BYTES); return (long) buf.getFloat(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return (double) buf.getFloat(position + Long.BYTES); return buf.getFloat(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

View File

@ -20,46 +20,28 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
public class LongLastAggregator implements Aggregator public class LongLastAggregator extends NumericLastAggregator<BaseLongColumnValueSelector>
{ {
private final BaseLongColumnValueSelector valueSelector; long lastValue;
private final BaseLongColumnValueSelector timeSelector;
protected long lastTime;
protected long lastValue;
public LongLastAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector) public LongLastAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
{ {
this.valueSelector = valueSelector; super(timeSelector, valueSelector);
this.timeSelector = timeSelector;
lastTime = Long.MIN_VALUE;
lastValue = 0; lastValue = 0;
} }
@Override @Override
public void aggregate() void setCurrentValue()
{ {
long time = timeSelector.getLong(); lastValue = valueSelector.getLong();
if (time >= lastTime) {
lastTime = time;
lastValue = valueSelector.getLong();
}
}
@Override
public double getDouble()
{
return (double) lastValue;
} }
@Override @Override
public Object get() public Object get()
{ {
return new SerializablePair<>(lastTime, lastValue); return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
} }
@Override @Override
@ -68,15 +50,15 @@ public class LongLastAggregator implements Aggregator
return (float) lastValue; return (float) lastValue;
} }
@Override
public double getDouble()
{
return (double) lastValue;
}
@Override @Override
public long getLong() public long getLong()
{ {
return lastValue; return lastValue;
} }
@Override
public void close()
{
}
} }

View File

@ -30,11 +30,12 @@ import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NullableNumericAggregatorFactory;
import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory; import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -46,8 +47,32 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
public class LongLastAggregatorFactory extends NullableNumericAggregatorFactory<ColumnValueSelector> public class LongLastAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new LongLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new LongLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
private final String fieldName; private final String fieldName;
private final String name; private final String name;
@ -64,21 +89,31 @@ public class LongLastAggregatorFactory extends NullableNumericAggregatorFactory<
} }
@Override @Override
protected ColumnValueSelector selector(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
return metricFactory.makeColumnValueSelector(fieldName); final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new LongLastAggregator(
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
valueSelector
);
}
} }
@Override @Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
return new LongLastAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
} if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
@Override } else {
protected BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) return new LongLastBufferAggregator(
{ metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
return new LongLastBufferAggregator(metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), selector); valueSelector
);
}
} }
@Override @Override
@ -118,35 +153,52 @@ public class LongLastAggregatorFactory extends NullableNumericAggregatorFactory<
return new LongLastAggregatorFactory(name, name) return new LongLastAggregatorFactory(name, name)
{ {
@Override @Override
public Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongLastAggregator(null, null) return new LongLastAggregator(null, null)
{ {
@Override @Override
public void aggregate() public void aggregate()
{ {
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject(); SerializablePair<Long, Long> pair = selector.getObject();
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
lastTime = pair.lhs; lastTime = pair.lhs;
lastValue = pair.rhs; if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
} }
} }
}; };
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongLastBufferAggregator(null, null) return new LongLastBufferAggregator(null, null)
{ {
@Override
public void putValue(ByteBuffer buf, int position)
{
SerializablePair<Long, Long> pair = selector.getObject();
buf.putLong(position, pair.rhs);
}
@Override @Override
public void aggregate(ByteBuffer buf, int position) public void aggregate(ByteBuffer buf, int position)
{ {
SerializablePair<Long, Long> pair = (SerializablePair<Long, Long>) selector.getObject(); SerializablePair<Long, Long> pair = selector.getObject();
long lastTime = buf.getLong(position); long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) { if (pair.lhs >= lastTime) {
buf.putLong(position, pair.lhs); if (pair.rhs != null) {
buf.putLong(position + Long.BYTES, pair.rhs); updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
} }
} }
@ -170,6 +222,9 @@ public class LongLastAggregatorFactory extends NullableNumericAggregatorFactory<
public Object deserialize(Object object) public Object deserialize(Object object)
{ {
Map map = (Map) object; Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue()); return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
} }
@ -213,13 +268,15 @@ public class LongLastAggregatorFactory extends NullableNumericAggregatorFactory<
@Override @Override
public String getTypeName() public String getTypeName()
{ {
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return "long"; return "long";
} }
@Override @Override
public int getMaxIntermediateSize() public int getMaxIntermediateSize()
{ {
return Long.BYTES * 2; // timestamp, is null, value
return Long.BYTES + Byte.BYTES + Long.BYTES;
} }
@Override @Override

View File

@ -20,75 +20,51 @@
package org.apache.druid.query.aggregation.last; package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair; import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class LongLastBufferAggregator implements BufferAggregator public class LongLastBufferAggregator extends NumericLastBufferAggregator<BaseLongColumnValueSelector>
{ {
private final BaseLongColumnValueSelector timeSelector;
private final BaseLongColumnValueSelector valueSelector;
public LongLastBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector) public LongLastBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
{ {
this.timeSelector = timeSelector; super(timeSelector, valueSelector);
this.valueSelector = valueSelector;
} }
@Override @Override
public void init(ByteBuffer buf, int position) void initValue(ByteBuffer buf, int position)
{ {
buf.putLong(position, Long.MIN_VALUE); buf.putLong(position, 0);
buf.putLong(position + Long.BYTES, 0);
} }
@Override @Override
public void aggregate(ByteBuffer buf, int position) void putValue(ByteBuffer buf, int position)
{ {
long time = timeSelector.getLong(); buf.putLong(position, valueSelector.getLong());
long lastTime = buf.getLong(position);
if (time >= lastTime) {
buf.putLong(position, time);
buf.putLong(position + Long.BYTES, valueSelector.getLong());
}
} }
@Override @Override
public Object get(ByteBuffer buf, int position) public Object get(ByteBuffer buf, int position)
{ {
return new SerializablePair<>(buf.getLong(position), buf.getLong(position + Long.BYTES)); boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
} }
@Override @Override
public float getFloat(ByteBuffer buf, int position) public float getFloat(ByteBuffer buf, int position)
{ {
return (float) buf.getLong(position + Long.BYTES); return (float) buf.getLong(position + VALUE_OFFSET);
} }
@Override @Override
public double getDouble(ByteBuffer buf, int position) public double getDouble(ByteBuffer buf, int position)
{ {
return buf.getLong(position + Long.BYTES); return buf.getLong(position + VALUE_OFFSET);
} }
@Override @Override
public long getLong(ByteBuffer buf, int position) public long getLong(ByteBuffer buf, int position)
{ {
return buf.getLong(position + Long.BYTES); return buf.getLong(position + VALUE_OFFSET);
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
} }
} }

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.query.aggregation.last;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
/**
* Base type for on heap 'last' aggregator for primitive numeric column selectors..
*
* This could probably share a base class with {@link org.apache.druid.query.aggregation.first.NumericFirstAggregator}
*/
public abstract class NumericLastAggregator<TSelector extends BaseNullableColumnValueSelector> implements Aggregator
{
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
long lastTime;
boolean rhsNull;
public NumericLastAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
lastTime = Long.MIN_VALUE;
rhsNull = !useDefault;
}
@Override
public void aggregate()
{
long time = timeSelector.getLong();
if (time >= lastTime) {
lastTime = time;
if (useDefault || !valueSelector.isNull()) {
setCurrentValue();
rhsNull = false;
} else {
rhsNull = true;
}
}
}
@Override
public void close()
{
// nothing to close
}
/**
* Store the current primitive typed 'first' value
*/
abstract void setCurrentValue();
}

View File

@ -0,0 +1,115 @@
/*
* 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.query.aggregation.last;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
import java.nio.ByteBuffer;
/**
* Base type for buffer based 'last' aggregator for primitive numeric column selectors
*
* This could probably share a base type with
* {@link org.apache.druid.query.aggregation.first.NumericFirstBufferAggregator} ...
*/
public abstract class NumericLastBufferAggregator<TSelector extends BaseNullableColumnValueSelector>
implements BufferAggregator
{
static final int NULL_OFFSET = Long.BYTES;
static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
public NumericLastBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
}
/**
* Initialize the buffer value at the position of {@link #VALUE_OFFSET}
*/
abstract void initValue(ByteBuffer buf, int position);
/**
* Place the primitive value in the buffer at the position of {@link #VALUE_OFFSET}
*/
abstract void putValue(ByteBuffer buf, int position);
boolean isValueNull(ByteBuffer buf, int position)
{
return buf.get(position + NULL_OFFSET) == NullHandling.IS_NULL_BYTE;
}
void updateTimeWithValue(ByteBuffer buf, int position, long time)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET);
}
void updateTimeWithNull(ByteBuffer buf, int position, long time)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NULL_BYTE);
}
@Override
public void init(ByteBuffer buf, int position)
{
buf.putLong(position, Long.MIN_VALUE);
buf.put(position + NULL_OFFSET, useDefault ? NullHandling.IS_NOT_NULL_BYTE : NullHandling.IS_NULL_BYTE);
initValue(buf, position + VALUE_OFFSET);
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
long time = timeSelector.getLong();
long lastTime = buf.getLong(position);
if (time >= lastTime) {
if (useDefault || !valueSelector.isNull()) {
updateTimeWithValue(buf, position, time);
} else {
updateTimeWithNull(buf, position, time);
}
}
}
@Override
public void close()
{
// no resources to cleanup
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("timeSelector", timeSelector);
inspector.visit("valueSelector", valueSelector);
}
}

View File

@ -65,7 +65,7 @@ public class StringLastAggregator implements Aggregator
valueSelector valueSelector
); );
if (inPair != null && inPair.rhs != null && inPair.lhs >= lastTime) { if (inPair != null && inPair.lhs >= lastTime) {
lastTime = inPair.lhs; lastTime = inPair.lhs;
lastValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes); lastValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes);
} }
@ -74,11 +74,8 @@ public class StringLastAggregator implements Aggregator
if (time >= lastTime) { if (time >= lastTime) {
final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject()); final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject());
lastTime = time;
if (value != null) { lastValue = StringUtils.fastLooseChop(value, maxStringBytes);
lastTime = time;
lastValue = StringUtils.fastLooseChop(value, maxStringBytes);
}
} }
} }
} }

View File

@ -36,9 +36,11 @@ import org.apache.druid.query.aggregation.first.StringFirstLastUtils;
import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
@ -48,6 +50,34 @@ import java.util.Objects;
@JsonTypeName("stringLast") @JsonTypeName("stringLast")
public class StringLastAggregatorFactory extends AggregatorFactory public class StringLastAggregatorFactory extends AggregatorFactory
{ {
private static final Aggregator NIL_AGGREGATOR = new StringLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance(),
0,
false
)
{
@Override
public void aggregate()
{
// no-op
}
};
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new StringLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance(),
0,
false
)
{
@Override
public void aggregate(ByteBuffer buf, int position)
{
// no-op
}
};
private final String fieldName; private final String fieldName;
private final String name; private final String name;
protected final int maxStringBytes; protected final int maxStringBytes;
@ -77,24 +107,32 @@ public class StringLastAggregatorFactory extends AggregatorFactory
public Aggregator factorize(ColumnSelectorFactory metricFactory) public Aggregator factorize(ColumnSelectorFactory metricFactory)
{ {
final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName); final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName);
return new StringLastAggregator( if (valueSelector instanceof NilColumnValueSelector) {
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), return NIL_AGGREGATOR;
valueSelector, } else {
maxStringBytes, return new StringLastAggregator(
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName)) metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
); valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
);
}
} }
@Override @Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{ {
final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName); final BaseObjectColumnValueSelector<?> valueSelector = metricFactory.makeColumnValueSelector(fieldName);
return new StringLastBufferAggregator( if (valueSelector instanceof NilColumnValueSelector) {
metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME), return NIL_BUFFER_AGGREGATOR;
valueSelector, } else {
maxStringBytes, return new StringLastBufferAggregator(
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName)) metricFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME),
); valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
);
}
} }
@Override @Override

View File

@ -72,7 +72,7 @@ public class StringLastBufferAggregator implements BufferAggregator
valueSelector valueSelector
); );
if (inPair != null && inPair.rhs != null) { if (inPair != null) {
final long lastTime = buf.getLong(position); final long lastTime = buf.getLong(position);
if (inPair.lhs >= lastTime) { if (inPair.lhs >= lastTime) {
StringFirstLastUtils.writePair( StringFirstLastUtils.writePair(
@ -90,14 +90,12 @@ public class StringLastBufferAggregator implements BufferAggregator
if (time >= lastTime) { if (time >= lastTime) {
final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject()); final String value = DimensionHandlerUtils.convertObjectToString(valueSelector.getObject());
if (value != null) { StringFirstLastUtils.writePair(
StringFirstLastUtils.writePair( buf,
buf, position,
position, new SerializablePairLongString(time, value),
new SerializablePairLongString(time, value), maxStringBytes
maxStringBytes );
);
}
} }
} }
} }

View File

@ -30,14 +30,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class DoubleFirstAggregationTest public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
{ {
private DoubleFirstAggregatorFactory doubleFirstAggFactory; private DoubleFirstAggregatorFactory doubleFirstAggFactory;
private DoubleFirstAggregatorFactory combiningAggFactory; private DoubleFirstAggregatorFactory combiningAggFactory;
@ -118,6 +120,31 @@ public class DoubleFirstAggregationTest
Assert.assertEquals(pair1, doubleFirstAggFactory.combine(pair1, pair2)); Assert.assertEquals(pair1, doubleFirstAggFactory.combine(pair1, pair2));
} }
@Test
public void testComparator()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621);
SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4);
Comparator comparator = doubleFirstAggFactory.getComparator();
Assert.assertEquals(-1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(1, comparator.compare(pair2, pair1));
}
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = doubleFirstAggFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testDoubleFirstCombiningAggregator() public void testDoubleFirstCombiningAggregator()
{ {

View File

@ -30,14 +30,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class FloatFirstAggregationTest public class FloatFirstAggregationTest extends InitializedNullHandlingTest
{ {
private FloatFirstAggregatorFactory floatFirstAggregatorFactory; private FloatFirstAggregatorFactory floatFirstAggregatorFactory;
private FloatFirstAggregatorFactory combiningAggFactory; private FloatFirstAggregatorFactory combiningAggFactory;
@ -65,8 +67,8 @@ public class FloatFirstAggregationTest
objectSelector = new TestObjectColumnSelector<>(pairs); objectSelector = new TestObjectColumnSelector<>(pairs);
colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class);
EasyMock.expect(colSelectorFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME)).andReturn(timeSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME)).andReturn(timeSelector);
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector).atLeastOnce();
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector).atLeastOnce();
EasyMock.replay(colSelectorFactory); EasyMock.replay(colSelectorFactory);
} }
@ -113,11 +115,23 @@ public class FloatFirstAggregationTest
@Test @Test
public void testCombine() public void testCombine()
{ {
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621); SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4); SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4f);
Assert.assertEquals(pair1, floatFirstAggregatorFactory.combine(pair1, pair2)); Assert.assertEquals(pair1, floatFirstAggregatorFactory.combine(pair1, pair2));
} }
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = floatFirstAggregatorFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testDoubleFirstCombiningAggregator() public void testDoubleFirstCombiningAggregator()
{ {

View File

@ -29,14 +29,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class LongFirstAggregationTest public class LongFirstAggregationTest extends InitializedNullHandlingTest
{ {
private LongFirstAggregatorFactory longFirstAggFactory; private LongFirstAggregatorFactory longFirstAggFactory;
private LongFirstAggregatorFactory combiningAggFactory; private LongFirstAggregatorFactory combiningAggFactory;
@ -117,6 +119,18 @@ public class LongFirstAggregationTest
Assert.assertEquals(pair1, longFirstAggFactory.combine(pair1, pair2)); Assert.assertEquals(pair1, longFirstAggFactory.combine(pair1, pair2));
} }
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 1263L);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = longFirstAggFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testLongFirstCombiningAggregator() public void testLongFirstCombiningAggregator()
{ {

View File

@ -189,7 +189,7 @@ public class StringFirstBufferAggregatorTest
SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position)); SerializablePairLongString sp = ((SerializablePairLongString) agg.get(buf, position));
Assert.assertEquals(1526724600L, (long) sp.lhs); Assert.assertEquals(1526724000L, (long) sp.lhs);
Assert.assertEquals("2.0", sp.rhs); Assert.assertEquals(null, sp.rhs);
} }
} }

View File

@ -43,6 +43,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -50,7 +51,7 @@ import org.junit.Test;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
public class StringFirstTimeseriesQueryTest public class StringFirstTimeseriesQueryTest extends InitializedNullHandlingTest
{ {
private static final String VISITOR_ID = "visitor_id"; private static final String VISITOR_ID = "visitor_id";
private static final String CLIENT_TYPE = "client_type"; private static final String CLIENT_TYPE = "client_type";

View File

@ -30,14 +30,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class DoubleLastAggregationTest public class DoubleLastAggregationTest extends InitializedNullHandlingTest
{ {
private DoubleLastAggregatorFactory doubleLastAggFactory; private DoubleLastAggregatorFactory doubleLastAggFactory;
private DoubleLastAggregatorFactory combiningAggFactory; private DoubleLastAggregatorFactory combiningAggFactory;
@ -118,6 +120,18 @@ public class DoubleLastAggregationTest
Assert.assertEquals(pair2, doubleLastAggFactory.combine(pair1, pair2)); Assert.assertEquals(pair2, doubleLastAggFactory.combine(pair1, pair2));
} }
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = doubleLastAggFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testDoubleLastCombiningAggregator() public void testDoubleLastCombiningAggregator()
{ {

View File

@ -30,14 +30,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class FloatLastAggregationTest public class FloatLastAggregationTest extends InitializedNullHandlingTest
{ {
private FloatLastAggregatorFactory floatLastAggregatorFactory; private FloatLastAggregatorFactory floatLastAggregatorFactory;
private FloatLastAggregatorFactory combiningAggFactory; private FloatLastAggregatorFactory combiningAggFactory;
@ -113,11 +115,23 @@ public class FloatLastAggregationTest
@Test @Test
public void testCombine() public void testCombine()
{ {
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621); SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4); SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4f);
Assert.assertEquals(pair2, floatLastAggregatorFactory.combine(pair1, pair2)); Assert.assertEquals(pair2, floatLastAggregatorFactory.combine(pair1, pair2));
} }
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = floatLastAggregatorFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testDoubleLastCombiningAggregator() public void testDoubleLastCombiningAggregator()
{ {

View File

@ -29,14 +29,16 @@ import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Comparator;
public class LongLastAggregationTest public class LongLastAggregationTest extends InitializedNullHandlingTest
{ {
private LongLastAggregatorFactory longLastAggFactory; private LongLastAggregatorFactory longLastAggFactory;
private LongLastAggregatorFactory combiningAggFactory; private LongLastAggregatorFactory combiningAggFactory;
@ -117,6 +119,18 @@ public class LongLastAggregationTest
Assert.assertEquals(pair2, longLastAggFactory.combine(pair1, pair2)); Assert.assertEquals(pair2, longLastAggFactory.combine(pair1, pair2));
} }
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 1263L);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
Comparator comparator = longLastAggFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
Assert.assertEquals(0, comparator.compare(pair2, pair2));
Assert.assertEquals(-1, comparator.compare(pair2, pair1));
}
@Test @Test
public void testLongLastCombiningAggregator() public void testLongLastCombiningAggregator()
{ {

View File

@ -1258,7 +1258,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build() .build()
), ),
ImmutableList.of( ImmutableList.of(
new Object[]{1L, 1.0f, NullHandling.sqlCompatible() ? "" : "10.1", 2L, 2.0f, "1"} new Object[]{1L, 1.0f, "", 2L, 2.0f, "1"}
) )
); );
} }
@ -1582,7 +1582,16 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.build() .build()
), ),
ImmutableList.of( ImmutableList.of(
new Object[]{NullHandling.sqlCompatible() ? 12.1 : 11.1} // default mode subquery results:
//[, 10.1]
//[a, ]
//[abc, def]
// sql compatible mode subquery results:
//[null, 10.1]
//[, 2]
//[a, ]
//[abc, def]
new Object[]{NullHandling.sqlCompatible() ? 12.1 : 10.1}
) )
); );
} }
@ -1673,6 +1682,392 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
); );
} }
@Test
public void testEarliestAggregatorsNumericNulls() throws Exception
{
// Cannot vectorize EARLIEST aggregator.
skipVectorize();
testQuery(
"SELECT EARLIEST(l1), EARLIEST(d1), EARLIEST(f1) FROM druid.numfoo",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
aggregators(
new LongFirstAggregatorFactory("a0", "l1"),
new DoubleFirstAggregatorFactory("a1", "d1"),
new FloatFirstAggregatorFactory("a2", "f1")
)
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{7L, 1.0, 1.0f}
)
);
}
@Test
public void testLatestAggregatorsNumericNull() throws Exception
{
// Cannot vectorize LATEST aggregator.
skipVectorize();
testQuery(
"SELECT LATEST(l1), LATEST(d1), LATEST(f1) FROM druid.numfoo",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
aggregators(
new LongLastAggregatorFactory("a0", "l1"),
new DoubleLastAggregatorFactory("a1", "d1"),
new FloatLastAggregatorFactory("a2", "f1")
)
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{NullHandling.defaultLongValue(), NullHandling.defaultDoubleValue(), NullHandling.defaultFloatValue()}
)
);
}
@Test
public void testFirstLatestAggregatorsSkipNulls() throws Exception
{
// Cannot vectorize LATEST aggregator.
skipVectorize();
final DimFilter filter;
if (useDefault) {
filter = not(selector("dim1", null, null));
} else {
filter = and(
not(selector("dim1", null, null)),
not(selector("l1", null, null)),
not(selector("d1", null, null)),
not(selector("f1", null, null))
);
}
testQuery(
"SELECT EARLIEST(dim1, 32), LATEST(l1), LATEST(d1), LATEST(f1) "
+ "FROM druid.numfoo "
+ "WHERE dim1 IS NOT NULL AND l1 IS NOT NULL AND d1 IS NOT NULL AND f1 is NOT NULL",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.filters(filter)
.aggregators(
aggregators(
new StringFirstAggregatorFactory("a0", "dim1", 32),
new LongLastAggregatorFactory("a1", "l1"),
new DoubleLastAggregatorFactory("a2", "d1"),
new FloatLastAggregatorFactory("a3", "f1")
)
)
.context(TIMESERIES_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
// first row of dim1 is empty string, which is null in default mode, last non-null numeric rows are zeros
new Object[]{useDefault ? "10.1" : "", 0L, 0.0, 0.0f}
)
);
}
@Test
public void testOrderByEarliestFloat() throws Exception
{
// Cannot vectorize EARLIEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0.0f},
new Object[]{"2", 0.0f},
new Object[]{"abc", 0.0f},
new Object[]{"def", 0.0f},
new Object[]{"10.1", 0.1f},
new Object[]{"", 1.0f}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0.0f},
new Object[]{"10.1", 0.1f},
new Object[]{"", 1.0f}
);
}
testQuery(
"SELECT dim1, EARLIEST(f1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new FloatFirstAggregatorFactory("a0", "f1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test
public void testOrderByEarliestDouble() throws Exception
{
// Cannot vectorize EARLIEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0.0},
new Object[]{"2", 0.0},
new Object[]{"abc", 0.0},
new Object[]{"def", 0.0},
new Object[]{"", 1.0},
new Object[]{"10.1", 1.7}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0.0},
new Object[]{"", 1.0},
new Object[]{"10.1", 1.7}
);
}
testQuery(
"SELECT dim1, EARLIEST(d1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new DoubleFirstAggregatorFactory("a0", "d1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test
public void testOrderByEarliestLong() throws Exception
{
// Cannot vectorize EARLIEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0L},
new Object[]{"2", 0L},
new Object[]{"abc", 0L},
new Object[]{"def", 0L},
new Object[]{"", 7L},
new Object[]{"10.1", 325323L}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0L},
new Object[]{"", 7L},
new Object[]{"10.1", 325323L}
);
}
testQuery(
"SELECT dim1, EARLIEST(l1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new LongFirstAggregatorFactory("a0", "l1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test
public void testOrderByLatestFloat() throws Exception
{
// Cannot vectorize LATEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0.0f},
new Object[]{"2", 0.0f},
new Object[]{"abc", 0.0f},
new Object[]{"def", 0.0f},
new Object[]{"10.1", 0.1f},
new Object[]{"", 1.0f}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0.0f},
new Object[]{"10.1", 0.1f},
new Object[]{"", 1.0f}
);
}
testQuery(
"SELECT dim1, LATEST(f1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new FloatLastAggregatorFactory("a0", "f1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test
public void testOrderByLatestDouble() throws Exception
{
// Cannot vectorize LATEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0.0},
new Object[]{"2", 0.0},
new Object[]{"abc", 0.0},
new Object[]{"def", 0.0},
new Object[]{"", 1.0},
new Object[]{"10.1", 1.7}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0.0},
new Object[]{"", 1.0},
new Object[]{"10.1", 1.7}
);
}
testQuery(
"SELECT dim1, LATEST(d1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new DoubleLastAggregatorFactory("a0", "d1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test
public void testOrderByLatestLong() throws Exception
{
// Cannot vectorize LATEST aggregator.
skipVectorize();
List<Object[]> expected;
if (NullHandling.replaceWithDefault()) {
expected = ImmutableList.of(
new Object[]{"1", 0L},
new Object[]{"2", 0L},
new Object[]{"abc", 0L},
new Object[]{"def", 0L},
new Object[]{"", 7L},
new Object[]{"10.1", 325323L}
);
} else {
expected = ImmutableList.of(
new Object[]{"1", null},
new Object[]{"abc", null},
new Object[]{"def", null},
new Object[]{"2", 0L},
new Object[]{"", 7L},
new Object[]{"10.1", 325323L}
);
}
testQuery(
"SELECT dim1, LATEST(l1) FROM druid.numfoo GROUP BY 1 ORDER BY 2 LIMIT 10",
ImmutableList.of(
new TopNQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.dimension(new DefaultDimensionSpec("dim1", "_d0"))
.aggregators(
aggregators(
new LongLastAggregatorFactory("a0", "l1")
)
)
.metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0")))
.threshold(10)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
expected
);
}
@Test @Test
public void testGroupByLong() throws Exception public void testGroupByLong() throws Exception
{ {