Handling review comments and correctness fix for latest_by when the time expression need not be in sorted order

This commit is contained in:
Soumyava Das 2023-08-16 15:43:21 -07:00
parent aa971815a8
commit f58541273b
9 changed files with 120 additions and 54 deletions

View File

@ -57,21 +57,21 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
{
final long[] timeVector = timeSelector.getLongVector();
final boolean[] nullTimeVector = timeSelector.getNullVector();
final boolean[] nullValueVector = valueSelector.getNullVector();
firstTime = buf.getLong(position);
// check if nullVector is found or not
// the nullVector is null if no null values are found
// set the nullAbsent flag accordingly
// the time vector is already sorted so the first element would be the earliest
// traverse accordingly
int index;
for (int i = startRow; i < endRow; i++) {
index = i;
// Now we are iterating over the values to find the minima as the
// timestamp expression in EARLIEST_BY has no established sorting order
// If we know that the time is already sorted this can be optimized
// for the general EARLIEST call which is always on __time which is sorted
for (index = startRow; index < endRow; index++) {
if (nullTimeVector != null && nullTimeVector[index]) {
continue;
}
final long earliestTime = timeVector[index];
if (earliestTime >= firstTime) {
break;
continue;
}
firstTime = earliestTime;
if (useDefault || nullValueVector == null || !nullValueVector[index]) {
@ -83,11 +83,10 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
}
/**
*
* Checks if the aggregated value at a position in the buffer is null or not
*
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @return
*/
boolean isValueNull(ByteBuffer buf, int position)
@ -110,7 +109,7 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
for (int i = 0; i < numRows; i++) {
int position = positions[i] + positionOffset;
int row = rows == null ? i : rows[i];
long firstTime = buf.getLong(position);
firstTime = buf.getLong(position);
if (timeVector[row] < firstTime) {
if (useDefault || nulls == null || !nulls[row]) {
updateTimeWithValue(buf, position, timeVector[row], row);
@ -124,10 +123,10 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
/**
* Updates the time and the non null values to the appropriate position in buffer
*
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param time the time to be updated in the buffer as the last time
* @param index the index of the vectorized vector which is the last value
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param time the time to be updated in the buffer as the last time
* @param index the index of the vectorized vector which is the last value
*/
void updateTimeWithValue(ByteBuffer buf, int position, long time, int index)
{
@ -139,9 +138,9 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
/**
* Updates the time only to the appropriate position in buffer as the value is null
*
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param time the time to be updated in the buffer as the last time
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param time the time to be updated in the buffer as the last time
*/
void updateTimeWithNull(ByteBuffer buf, int position, long time)
{
@ -150,7 +149,7 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
}
/**
*Abstract function which needs to be overridden by subclasses to set the initial value
* Abstract function which needs to be overridden by subclasses to set the initial value
*/
abstract void initValue(ByteBuffer buf, int position);

View File

@ -66,14 +66,21 @@ public class SingleStringFirstDimensionVectorAggregator implements VectorAggrega
final long[] timeVector = timeSelector.getLongVector();
final int[] valueVector = valueDimensionVectorSelector.getRowVector();
firstTime = buf.getLong(position);
int index = startRow;
int index;
long earliestTime;
final long earliestTime = timeVector[index];
if (earliestTime < firstTime) {
firstTime = earliestTime;
buf.putLong(position, firstTime);
buf.put(position + NumericFirstVectorAggregator.NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
buf.putInt(position + NumericFirstVectorAggregator.VALUE_OFFSET, valueVector[index]);
// Now we are iterating over the values to find the minima as the
// timestamp expression in EARLIEST_BY has no established sorting order
// If we know that the time is already sorted this can be optimized
// for the general EARLIEST call which is always on __time which is sorted
for (index = startRow; index < endRow; index++) {
earliestTime = timeVector[index];
if (earliestTime < firstTime) {
firstTime = earliestTime;
buf.putLong(position, firstTime);
buf.put(position + NumericFirstVectorAggregator.NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
buf.putInt(position + NumericFirstVectorAggregator.VALUE_OFFSET, valueVector[index]);
}
}
}
@ -81,19 +88,45 @@ public class SingleStringFirstDimensionVectorAggregator implements VectorAggrega
public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
{
long[] timeVector = timeSelector.getLongVector();
boolean[] nullTimeVector = timeSelector.getNullVector();
int[] values = valueDimensionVectorSelector.getRowVector();
for (int i = 0; i < numRows; i++) {
int position = positions[i] + positionOffset;
int row = rows == null ? i : rows[i];
long firstTime = buf.getLong(position);
if (timeVector[row] < firstTime) {
firstTime = timeVector[row];
buf.putLong(position, firstTime);
buf.put(position + NumericFirstVectorAggregator.NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
buf.putInt(position + NumericFirstVectorAggregator.VALUE_OFFSET, values[row]);
// Now we are iterating over the values to find the minima as the
// timestamp expression in EARLIEST_BY has no established sorting order
// If we know that the time is already sorted this can be optimized
// for the general EARLIEST call which is always on __time which is sorted
// The hotpath is separated out into 2 cases when nullTimeVector
// is null and not-null so that the check is not on every value
if (nullTimeVector != null) {
for (int i = 0; i < numRows; i++) {
if (nullTimeVector[i]) {
continue;
}
int position = positions[i] + positionOffset;
int row = rows == null ? i : rows[i];
long firstTime = buf.getLong(position);
if (timeVector[row] < firstTime) {
firstTime = timeVector[row];
buf.putLong(position, firstTime);
buf.put(position + NumericFirstVectorAggregator.NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
buf.putInt(position + NumericFirstVectorAggregator.VALUE_OFFSET, values[row]);
}
}
} else {
for (int i = 0; i < numRows; i++) {
int position = positions[i] + positionOffset;
int row = rows == null ? i : rows[i];
long firstTime = buf.getLong(position);
if (timeVector[row] < firstTime) {
firstTime = timeVector[row];
buf.putLong(position, firstTime);
buf.put(position + NumericFirstVectorAggregator.NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
buf.putInt(position + NumericFirstVectorAggregator.VALUE_OFFSET, values[row]);
}
}
}
}
@Nullable

View File

@ -56,9 +56,6 @@ public class StringFirstAggregator implements Aggregator
@Override
public void aggregate()
{
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
// Less efficient code path when folding is a possibility (we must read the value selector first just in case
// it's a foldable object).
@ -72,6 +69,9 @@ public class StringFirstAggregator implements Aggregator
firstValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes);
}
} else {
if (timeSelector.isNull()) {
return;
}
final long time = timeSelector.getLong();
if (time < firstTime) {

View File

@ -63,9 +63,6 @@ public class StringFirstBufferAggregator implements BufferAggregator
@Override
public void aggregate(ByteBuffer buf, int position)
{
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
// Less efficient code path when folding is a possibility (we must read the value selector first just in case
// it's a foldable object).
@ -86,6 +83,9 @@ public class StringFirstBufferAggregator implements BufferAggregator
}
}
} else {
if (timeSelector.isNull()) {
return;
}
final long time = timeSelector.getLong();
final long firstTime = buf.getLong(position);

View File

@ -120,6 +120,9 @@ public class StringFirstLastUtils
time = pair.lhs;
string = pair.rhs;
} else if (object != null) {
if (timeSelector.isNull()) {
return null;
}
time = timeSelector.getLong();
string = DimensionHandlerUtils.convertObjectToString(object);
} else {

View File

@ -68,11 +68,14 @@ public class StringFirstVectorAggregator implements VectorAggregator
Object[] objectsWhichMightBeStrings = valueSelector.getObjectVector();
long firstTime = buf.getLong(position);
int index;
for (int i = startRow; i < endRow; i++) {
if (times[i] > firstTime) {
break;
// Now we are iterating over the values to find the minima as the
// timestamp expression in EARLIEST_BY has no established sorting order
// If we know that the time is already sorted this can be optimized
// for the general EARLIEST call which is always on __time which is sorted
for (index = startRow; index < endRow; index++) {
if (times[index] > firstTime) {
continue;
}
index = i;
final boolean foldNeeded = StringFirstLastUtils.objectNeedsFoldCheck(objectsWhichMightBeStrings[index]);
if (foldNeeded) {
final SerializablePairLongString inPair = StringFirstLastUtils.readPairFromVectorSelectorsAtIndex(
@ -132,6 +135,7 @@ public class StringFirstVectorAggregator implements VectorAggregator
long firstTime = buf.getLong(position);
if (timeVector[row] < firstTime) {
if (foldNeeded) {
firstTime = timeVector[row];
final SerializablePairLongString inPair = StringFirstLastUtils.readPairFromVectorSelectorsAtIndex(
timeSelector,
valueSelector,

View File

@ -57,9 +57,6 @@ public class StringLastAggregator implements Aggregator
@Override
public void aggregate()
{
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
// Less efficient code path when folding is a possibility (we must read the value selector first just in case
// it's a foldable object).
@ -73,6 +70,9 @@ public class StringLastAggregator implements Aggregator
lastValue = StringUtils.fastLooseChop(inPair.rhs, maxStringBytes);
}
} else {
if (timeSelector.isNull()) {
return;
}
final long time = timeSelector.getLong();
if (time >= lastTime) {

View File

@ -64,9 +64,6 @@ public class StringLastBufferAggregator implements BufferAggregator
@Override
public void aggregate(ByteBuffer buf, int position)
{
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
// Less efficient code path when folding is a possibility (we must read the value selector first just in case
// it's a foldable object).
@ -87,6 +84,9 @@ public class StringLastBufferAggregator implements BufferAggregator
}
}
} else {
if (timeSelector.isNull()) {
return;
}
final long time = timeSelector.getLong();
final long lastTime = buf.getLong(position);

View File

@ -707,6 +707,33 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@Test
public void testEarliestVectorAggregators()
{
notMsqCompatible();
testQuery(
"SELECT "
+ "EARLIEST(dim1, 10) "
+ "FROM druid.numfoo",
ImmutableList.of(
Druids.newTimeseriesQueryBuilder()
.dataSource(CalciteTests.DATASOURCE3)
.intervals(querySegmentSpec(Filtration.eternity()))
.granularity(Granularities.ALL)
.aggregators(
aggregators(
new StringFirstAggregatorFactory("a0", "dim1", "__time", 10)
)
)
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
ImmutableList.of(
new Object[]{""}
)
);
}
@Test
public void testLatestAggregators()
{