Add experimental support for first/last for double/float/long #10702 (#14462)

Add experimental support for doubleLast, doubleFirst, FloatLast, FloatFirst, longLast and longFirst.
This commit is contained in:
Ankit Kothari 2023-12-11 22:06:51 -08:00 committed by GitHub
parent 85af2c8340
commit 8735d023a1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
111 changed files with 6723 additions and 1313 deletions

View File

@ -26,6 +26,36 @@
"type":"stringLast",
"name":"latest_user",
"fieldName":"last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longFirst",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
]
},
@ -35,7 +65,13 @@
"event" : {
"continent":"Asia",
"earliest_user":"masterYi",
"latest_user":"stringer"
"latest_user":"stringer",
"double_first_delta": 111.0,
"double_last_delta": -9.0,
"long_first_delta": 111,
"long_last_delta": -9,
"float_first_delta": 111.0,
"float_last_delta": -9.0
}
} ]
}

View File

@ -40,6 +40,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "delta"
}
],
"granularitySpec": {

View File

@ -56,6 +56,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
]
}

View File

@ -37,6 +37,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
],
"granularitySpec": {

View File

@ -26,6 +26,36 @@
"type":"stringLast",
"name":"latest_user",
"fieldName":"last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longFirst",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
]
},
@ -35,7 +65,13 @@
"event" : {
"continent":"Asia",
"earliest_user":"masterYi",
"latest_user":"stringer"
"latest_user":"stringer",
"double_first_delta": 111.0,
"double_last_delta": -9.0,
"long_first_delta": 111,
"long_last_delta": -9,
"float_first_delta": 111.0,
"float_last_delta": -9.0
}
} ]
}

View File

@ -40,6 +40,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "delta"
}
],
"granularitySpec": {

View File

@ -56,6 +56,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
]
}

View File

@ -37,6 +37,36 @@
"type": "stringLast",
"name": "last_user",
"fieldName": "last_user"
},
{
"type": "doubleFirst",
"name": "double_first_delta",
"fieldName": "double_first_delta"
},
{
"type": "doubleLast",
"name": "double_last_delta",
"fieldName": "double_last_delta"
},
{
"type": "longFirst",
"name": "long_first_delta",
"fieldName": "long_first_delta"
},
{
"type": "longLast",
"name": "long_last_delta",
"fieldName": "long_last_delta"
},
{
"type": "floatFirst",
"name": "float_first_delta",
"fieldName": "float_first_delta"
},
{
"type": "floatLast",
"name": "float_last_delta",
"fieldName": "float_last_delta"
}
],
"granularitySpec": {

View File

@ -39,6 +39,9 @@ import org.apache.druid.query.aggregation.LongMaxAggregatorFactory;
import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongDoubleComplexMetricSerde;
import org.apache.druid.query.aggregation.SerializablePairLongFloatComplexMetricSerde;
import org.apache.druid.query.aggregation.SerializablePairLongLongComplexMetricSerde;
import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde;
import org.apache.druid.query.aggregation.any.DoubleAnyAggregatorFactory;
import org.apache.druid.query.aggregation.any.FloatAnyAggregatorFactory;
@ -83,6 +86,10 @@ public class AggregatorsModule extends SimpleModule
ComplexMetrics.registerSerde(PreComputedHyperUniquesSerde.TYPE_NAME, new PreComputedHyperUniquesSerde());
ComplexMetrics.registerSerde(SerializablePairLongStringComplexMetricSerde.TYPE_NAME, new SerializablePairLongStringComplexMetricSerde());
ComplexMetrics.registerSerde(SerializablePairLongFloatComplexMetricSerde.TYPE_NAME, new SerializablePairLongFloatComplexMetricSerde());
ComplexMetrics.registerSerde(SerializablePairLongDoubleComplexMetricSerde.TYPE_NAME, new SerializablePairLongDoubleComplexMetricSerde());
ComplexMetrics.registerSerde(SerializablePairLongLongComplexMetricSerde.TYPE_NAME, new SerializablePairLongLongComplexMetricSerde());
setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class);
setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class);

View File

@ -0,0 +1,61 @@
/*
* 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;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.segment.serde.ComplexMetricExtractor;
import org.apache.druid.segment.serde.ComplexMetricSerde;
import javax.annotation.Nullable;
// To maintain parity with the longString pair serde,
// we have made EXPECTED_VERSION as 3 because it uses the same delta & block encoding technique.
// 0,1,2 versions for numeric serdes do not exist.
public abstract class AbstractSerializableLongObjectPairSerde<T extends SerializablePair<Long, ?>> extends
ComplexMetricSerde
{
private final Class<T> pairClass;
AbstractSerializableLongObjectPairSerde(Class<T> pairClass)
{
this.pairClass = pairClass;
}
@Override
public ComplexMetricExtractor<?> getExtractor()
{
return new ComplexMetricExtractor<Object>()
{
@Override
public Class<T> extractedClass()
{
return pairClass;
}
@Nullable
@Override
public Object extractValue(InputRow inputRow, String metricName)
{
return inputRow.getRaw(metricName);
}
};
}
}

View File

@ -0,0 +1,139 @@
/*
* 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;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.segment.serde.Serializer;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.CellWriter;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
public abstract class AbstractSerializablePairLongObjectBufferStore<T extends SerializablePair<Long, ?>>
{
private final SerializedStorage<T> serializedStorage;
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
AbstractSerializablePairLongObjectBufferStore(SerializedStorage<T> serializedStorage)
{
this.serializedStorage = serializedStorage;
}
public void store(@Nullable T pairLongObject) throws IOException
{
if (pairLongObject != null && pairLongObject.lhs != null) {
minValue = Math.min(minValue, pairLongObject.lhs);
maxValue = Math.max(maxValue, pairLongObject.lhs);
}
serializedStorage.store(pairLongObject);
}
/**
* each call transfers the temporary buffer into an encoded, block-compessed buffer of the segment. It is ready to be
* transferred to a {@link WritableByteChannel}
*
* @param byteBufferProvider - provides a ByteBuffer used for block compressed encoding
* @param segmentWriteOutMedium - used to create temporary storage
* @return encoded buffer ready to be stored
* @throws IOException
*/
public TransferredBuffer transferToRowWriter(
ByteBufferProvider byteBufferProvider,
SegmentWriteOutMedium segmentWriteOutMedium
) throws IOException
{
AbstractSerializablePairLongObjectColumnHeader<T> columnHeader = createColumnHeader();
AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<T> deltaEncodedSerde = createDeltaEncodedSerde(columnHeader);
// try-with-resources will call cellWriter.close() an extra time in the normal case, but it protects against
// buffer leaking in the case of an exception (close() is idempotent). In the normal path, close() performs some
// finalization of the CellWriter object. We want that object state finalized before creating the TransferredBuffer
// as a point of good style (though strictly speaking, it works fine to pass it in before calling close since
// TransferredBuffer does not do anything in the constructor with the object)
try (CellWriter cellWriter = new CellWriter.Builder(segmentWriteOutMedium).setByteBufferProvider(byteBufferProvider)
.build()) {
try (IOIterator<T> bufferIterator = iterator()) {
while (bufferIterator.hasNext()) {
T pairLongObject = bufferIterator.next();
byte[] serialized = deltaEncodedSerde.serialize(pairLongObject);
cellWriter.write(serialized);
}
cellWriter.close();
return new TransferredBuffer(cellWriter, columnHeader);
}
}
}
// 1. we have overflow in our range || 2. we have only seen null values
// in this case, effectively disable delta encoding by using longs and a min value of 0
// else we shoudl return columnHeader with delta encding enabled
@Nonnull
public abstract AbstractSerializablePairLongObjectColumnHeader<T> createColumnHeader();
public abstract AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<T> createDeltaEncodedSerde(AbstractSerializablePairLongObjectColumnHeader<T> columnHeader);
public IOIterator<T> iterator() throws IOException
{
return serializedStorage.iterator();
}
/**
* contains serialized data that is compressed and delta-encoded (Long)
* It's ready to be transferred to a {@link WritableByteChannel}
*/
public static class TransferredBuffer implements Serializer
{
private final CellWriter cellWriter;
private final AbstractSerializablePairLongObjectColumnHeader<?> columnHeader;
public TransferredBuffer(
CellWriter cellWriter,
AbstractSerializablePairLongObjectColumnHeader<?> columnHeader
)
{
this.cellWriter = cellWriter;
this.columnHeader = columnHeader;
}
@Override
public long getSerializedSize()
{
return columnHeader.getSerializedSize() + cellWriter.getSerializedSize();
}
@Override
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
columnHeader.transferTo(channel);
cellWriter.writeTo(channel, smoosher);
}
}
}

View File

@ -0,0 +1,125 @@
/*
* 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;
import com.google.common.base.MoreObjects;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.segment.serde.cell.LongSerializer;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
import java.util.Locale;
public abstract class AbstractSerializablePairLongObjectColumnHeader<T extends SerializablePair<Long, ?>>
{
// header size is 4 bytes for word alignment for LZ4 (minmatch) compression
private static final int HEADER_SIZE_BYTES = 4;
private static final int USE_INTEGER_MASK = 0x80;
private static final int VERSION_INDEX = 0;
private static final int ENCODING_INDEX = 1;
private final byte[] bytes;
final long minValue;
AbstractSerializablePairLongObjectColumnHeader(byte[] bytes, long minValue)
{
this.bytes = bytes;
this.minValue = minValue;
}
AbstractSerializablePairLongObjectColumnHeader(int version, boolean useIntegerDeltas, long minTimestamp)
{
this.minValue = minTimestamp;
bytes = new byte[HEADER_SIZE_BYTES];
Preconditions.checkArgument(version <= 255, "max version 255");
bytes[VERSION_INDEX] = (byte) version;
if (useIntegerDeltas) {
bytes[ENCODING_INDEX] |= USE_INTEGER_MASK;
}
}
static AbstractSerializablePairLongObjectColumnHeader<?> fromBuffer(ByteBuffer byteBuffer, Class<?> pairClass)
{
byte[] bytes = new byte[HEADER_SIZE_BYTES];
byteBuffer.get(bytes);
long minTimestamp = byteBuffer.getLong();
if (pairClass.isAssignableFrom(SerializablePairLongLong.class)) {
return new SerializablePairLongLongColumnHeader(bytes, minTimestamp);
}
if (pairClass.isAssignableFrom(SerializablePairLongDouble.class)) {
return new SerializablePairLongDoubleColumnHeader(bytes, minTimestamp);
}
if (pairClass.isAssignableFrom(SerializablePairLongFloat.class)) {
return new SerializablePairLongFloatColumnHeader(bytes, minTimestamp);
}
if (pairClass.isAssignableFrom(SerializablePairLongString.class)) {
return new SerializablePairLongStringColumnHeader(bytes, minTimestamp);
}
throw new RE(String.format(Locale.ENGLISH, "Unsupported pairClass type: %s", pairClass.getSimpleName()));
}
public abstract AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> createSerde();
public void transferTo(WritableByteChannel channel) throws IOException
{
LongSerializer longSerializer = new LongSerializer();
channel.write(ByteBuffer.wrap(bytes));
channel.write(longSerializer.serialize(minValue));
}
public int getVersion()
{
return 0XFF & bytes[VERSION_INDEX];
}
public boolean isUseIntegerDeltas()
{
return (bytes[ENCODING_INDEX] & USE_INTEGER_MASK) != 0;
}
public long getMinValue()
{
return minValue;
}
public int getSerializedSize()
{
return HEADER_SIZE_BYTES + Long.BYTES;
}
@Override
public String toString()
{
return MoreObjects.toStringHelper(this)
.add("bytes", bytes)
.add("minValue", minValue)
.toString();
}
}

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;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
/**
* valid call sequence
* <p>
* open()+serialize()*(getSerializedSize()|writeTo())*
* <p>
* getSerializedSize() / writeTo() effectively function as a close call, but each may be called multiple times and has
* no effect on one another.
*/
@SuppressWarnings("NotNullFieldNotInitialized")
public abstract class AbstractSerializablePairLongObjectColumnSerializer<T extends SerializablePair<Long, ?>> implements
GenericColumnSerializer<T>
{
public final StagedSerde<T> stagedSerde;
final SegmentWriteOutMedium segmentWriteOutMedium;
private final ByteBufferProvider byteBufferProvider;
State state = State.START;
AbstractSerializablePairLongObjectBufferStore<T> bufferStore;
private AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer;
AbstractSerializablePairLongObjectColumnSerializer(
StagedSerde<T> stagedSerde,
SegmentWriteOutMedium segmentWriteOutMedium,
ByteBufferProvider byteBufferProvider
)
{
this.stagedSerde = stagedSerde;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.byteBufferProvider = byteBufferProvider;
}
@Override
public void serialize(ColumnValueSelector<? extends T> selector) throws IOException
{
Preconditions.checkState(state == State.OPEN, "serialize called in invalid state %s", state);
T pairLongObject = selector.getObject();
bufferStore.store(pairLongObject);
}
@Override
public long getSerializedSize() throws IOException
{
Preconditions.checkState(
state != State.START,
"getSerializedSize called in invalid state %s (must have opened at least)",
state
);
transferToRowWriterIfNecessary();
return transferredBuffer.getSerializedSize();
}
@Override
public void writeTo(
WritableByteChannel channel,
FileSmoosher smoosher
) throws IOException
{
Preconditions.checkState(state != State.START, "writeTo called in invalid state %s", state);
transferToRowWriterIfNecessary();
transferredBuffer.writeTo(channel, smoosher);
}
private void transferToRowWriterIfNecessary() throws IOException
{
if (state == State.OPEN) {
transferredBuffer = bufferStore.transferToRowWriter(byteBufferProvider, segmentWriteOutMedium);
state = State.CLOSED;
}
}
enum State
{
START,
OPEN,
CLOSED,
}
}

View File

@ -0,0 +1,119 @@
/*
* 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;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.serde.cell.StorableBuffer;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Locale;
/**
* serializes a Long/Object(Number) pair in the context of a column/segment. Uses the minValue to perform delta
* encoding/decoding and if the range of the segment fits in an integer (useIntegerDelta), the format is
* Integer:Byte:Integer
*
* otherwise
* Long:Integer:bytes
*/
public abstract class AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<T extends SerializablePair<Long, ?>> implements
StagedSerde<T>
{
final long minValue;
final boolean useIntegerDelta;
private final Class<?> pairClass;
AbstractSerializablePairLongObjectDeltaEncodedStagedSerde(
long minValue,
boolean useIntegerDelta,
Class<?> pairClass
)
{
this.minValue = minValue;
this.useIntegerDelta = useIntegerDelta;
this.pairClass = pairClass;
}
@Override
public StorableBuffer serializeDelayed(@Nullable T value)
{
if (value == null) {
return StorableBuffer.EMPTY;
}
Object rhsObject = value.getRhs();
return new StorableBuffer()
{
@Override
public void store(ByteBuffer byteBuffer)
{
Preconditions.checkNotNull(value.lhs, String.format(Locale.ENGLISH, "Long in %s must be non-null", pairClass.getSimpleName()));
long delta = value.lhs - minValue;
Preconditions.checkState(delta >= 0 || delta == value.lhs);
if (useIntegerDelta) {
byteBuffer.putInt(Ints.checkedCast(delta));
} else {
byteBuffer.putLong(delta);
}
if (rhsObject != null) {
byteBuffer.put(NullHandling.IS_NOT_NULL_BYTE);
if (pairClass.isAssignableFrom(SerializablePairLongLong.class)) {
byteBuffer.putLong((long) rhsObject);
} else if (pairClass.isAssignableFrom(SerializablePairLongDouble.class)) {
byteBuffer.putDouble((double) rhsObject);
} else if (pairClass.isAssignableFrom(SerializablePairLongFloat.class)) {
byteBuffer.putFloat((float) rhsObject);
}
} else {
byteBuffer.put(NullHandling.IS_NULL_BYTE);
}
}
@Override
public int getSerializedSize()
{
int rhsBytes = 0;
if (rhsObject != null) {
if (pairClass.isAssignableFrom(SerializablePairLongLong.class)) {
rhsBytes = Long.BYTES;
} else if (pairClass.isAssignableFrom(SerializablePairLongDouble.class)) {
rhsBytes = Double.BYTES;
} else if (pairClass.isAssignableFrom(SerializablePairLongFloat.class)) {
rhsBytes = Float.BYTES;
}
}
return (useIntegerDelta ? Integer.BYTES : Long.BYTES) + Byte.BYTES + rhsBytes;
}
};
}
}

View File

@ -0,0 +1,98 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.serde.cell.StorableBuffer;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
/**
* serializes a Long/Object pair as
* Long:Byte:Object
* <p>
* or
* Long:isNullByte:ObjectBytes
*/
public abstract class AbstractSerializablePairLongObjectSimpleStagedSerde<T extends SerializablePair<Long, ?>> implements StagedSerde<T>
{
private final Class<?> pairCLass;
AbstractSerializablePairLongObjectSimpleStagedSerde(Class<?> pairCLass)
{
this.pairCLass = pairCLass;
}
@Override
public StorableBuffer serializeDelayed(
@Nullable T value
)
{
if (value == null) {
return StorableBuffer.EMPTY;
}
Object rhsObject = value.getRhs();
return new StorableBuffer()
{
@Override
public void store(ByteBuffer byteBuffer)
{
Preconditions.checkNotNull(value.getLhs(), "Long in %s must be non-null", pairCLass.getSimpleName());
byteBuffer.putLong(value.getLhs());
if (rhsObject != null) {
byteBuffer.put(NullHandling.IS_NOT_NULL_BYTE);
if (pairCLass.isAssignableFrom(SerializablePairLongLong.class)) {
byteBuffer.putLong((long) rhsObject);
} else if (pairCLass.isAssignableFrom(SerializablePairLongDouble.class)) {
byteBuffer.putDouble((double) rhsObject);
} else if (pairCLass.isAssignableFrom(SerializablePairLongFloat.class)) {
byteBuffer.putFloat((float) rhsObject);
}
} else {
byteBuffer.put(NullHandling.IS_NULL_BYTE);
}
}
@Override
public int getSerializedSize()
{
int rhsBytes = 0;
if (rhsObject != null) {
if (pairCLass.isAssignableFrom(SerializablePairLongLong.class)) {
rhsBytes = Long.BYTES;
} else if (pairCLass.isAssignableFrom(SerializablePairLongDouble.class)) {
rhsBytes = Double.BYTES;
} else if (pairCLass.isAssignableFrom(SerializablePairLongFloat.class)) {
rhsBytes = Float.BYTES;
}
}
return Long.BYTES + Byte.BYTES + rhsBytes;
}
};
}
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.collections.SerializablePair;
import javax.annotation.Nullable;
public class SerializablePairLongDouble extends SerializablePair<Long, Double>
{
@JsonCreator
public SerializablePairLongDouble(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") @Nullable Double rhs)
{
super(lhs, rhs);
}
}

View File

@ -0,0 +1,66 @@
/*
* 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;
import javax.annotation.Nonnull;
public class SerializablePairLongDoubleBufferStore extends AbstractSerializablePairLongObjectBufferStore<SerializablePairLongDouble>
{
public SerializablePairLongDoubleBufferStore(SerializedStorage<SerializablePairLongDouble> serializedStorage)
{
super(serializedStorage);
}
@Override
@Nonnull
public AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongDouble> createColumnHeader()
{
long maxDelta = maxValue - minValue;
SerializablePairLongDoubleColumnHeader columnHeader;
if (minValue < maxValue && maxDelta < 0 || minValue > maxValue) {
maxDelta = Long.MAX_VALUE;
minValue = 0;
}
if (maxDelta <= Integer.MAX_VALUE) {
columnHeader = new SerializablePairLongDoubleColumnHeader(
SerializablePairLongDoubleComplexMetricSerde.EXPECTED_VERSION,
true,
minValue
);
} else {
columnHeader = new SerializablePairLongDoubleColumnHeader(
SerializablePairLongDoubleComplexMetricSerde.EXPECTED_VERSION,
false,
minValue
);
}
return columnHeader;
}
@Override
public AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongDouble> createDeltaEncodedSerde(AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongDouble> columnHeader)
{
return new SerializablePairLongDoubleDeltaEncodedStagedSerde(minValue, columnHeader.isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,39 @@
/*
* 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;
public class SerializablePairLongDoubleColumnHeader extends AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongDouble>
{
SerializablePairLongDoubleColumnHeader(byte[] bytes, long minValue)
{
super(bytes, minValue);
}
SerializablePairLongDoubleColumnHeader(int version, boolean useIntegerDeltas, long minTimestamp)
{
super(version, useIntegerDeltas, minTimestamp);
}
@Override
public SerializablePairLongDoubleDeltaEncodedStagedSerde createSerde()
{
return new SerializablePairLongDoubleDeltaEncodedStagedSerde(minValue, isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,51 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
public class SerializablePairLongDoubleColumnSerializer extends AbstractSerializablePairLongObjectColumnSerializer<SerializablePairLongDouble>
{
public SerializablePairLongDoubleColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
ByteBufferProvider byteBufferProvider
)
{
super(new SerializablePairLongDoubleSimpleStagedSerde(), segmentWriteOutMedium, byteBufferProvider);
}
@Override
public void open() throws IOException
{
Preconditions.checkState(state == State.START || state == State.OPEN, "open called in invalid state %s", state);
if (state == State.START) {
bufferStore = new SerializablePairLongDoubleBufferStore(
new SerializedStorage<>(segmentWriteOutMedium.makeWriteOutBytes(), stagedSerde)
);
state = State.OPEN;
}
}
}

View File

@ -0,0 +1,129 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.CellReader;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongDoubleComplexColumn implements ComplexColumn
{
private final Closer closer;
private final int serializedSize;
private final CellReader cellReader;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
public SerializablePairLongDoubleComplexColumn(
CellReader cellReader,
AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde,
Closer closer,
int serializedSize
)
{
this.cellReader = cellReader;
this.serde = serde;
this.closer = closer;
this.serializedSize = serializedSize;
}
@Override
public Class<?> getClazz()
{
return SerializablePairLongDouble.class;
}
@Override
public String getTypeName()
{
return SerializablePairLongDoubleComplexMetricSerde.TYPE_NAME;
}
@Override
public Object getRowValue(int rowNum)
{
return serde.deserialize(cellReader.getCell(rowNum));
}
@Override
public int getLength()
{
return serializedSize;
}
@Override
public void close()
{
try {
closer.close();
}
catch (IOException e) {
throw new RE(e, "error closing " + getClass().getName());
}
}
public static class Builder
{
private final int serializedSize;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
private final CellReader.Builder cellReaderBuilder;
public Builder(ByteBuffer buffer)
{
ByteBuffer masterByteBuffer = buffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
serializedSize = masterByteBuffer.remaining();
AbstractSerializablePairLongObjectColumnHeader<?> columnHeader =
AbstractSerializablePairLongObjectColumnHeader.fromBuffer(masterByteBuffer, SerializablePairLongDouble.class);
Preconditions.checkArgument(
columnHeader.getVersion() == SerializablePairLongDoubleComplexMetricSerde.EXPECTED_VERSION,
"version %s expected, got %s",
SerializablePairLongDoubleComplexMetricSerde.EXPECTED_VERSION,
columnHeader.getVersion()
);
serde = columnHeader.createSerde();
cellReaderBuilder = new CellReader.Builder(masterByteBuffer);
}
public Builder setByteBufferProvier(ByteBufferProvider byteBufferProvider)
{
cellReaderBuilder.setByteBufferProvider(byteBufferProvider);
return this;
}
public SerializablePairLongDoubleComplexColumn build()
{
Closer closer = Closer.create();
CellReader cellReader = cellReaderBuilder.build();
closer.register(cellReader);
return new SerializablePairLongDoubleComplexColumn(cellReader, serde, closer, serializedSize);
}
}
}

View File

@ -0,0 +1,109 @@
/*
* 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;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Comparator;
public class SerializablePairLongDoubleComplexMetricSerde extends AbstractSerializableLongObjectPairSerde<SerializablePairLongDouble>
{
public static final int EXPECTED_VERSION = 3;
public static final String TYPE_NAME = "serializablePairLongDouble";
private static final SerializablePairLongDoubleSimpleStagedSerde SERDE = new SerializablePairLongDoubleSimpleStagedSerde();
private static final Comparator<SerializablePair<Long, Double>> COMPARATOR = SerializablePair.createNullHandlingComparator(
Double::compare,
true
);
public SerializablePairLongDoubleComplexMetricSerde()
{
super(SerializablePairLongDouble.class);
}
@Override
public String getTypeName()
{
return TYPE_NAME;
}
@Override
public GenericColumnSerializer<SerializablePairLongDouble> getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return new SerializablePairLongDoubleColumnSerializer(
segmentWriteOutMedium,
NativeClearedByteBufferProvider.INSTANCE
);
}
@Override
public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
{
SerializablePairLongDoubleComplexColumn.Builder builder =
new SerializablePairLongDoubleComplexColumn.Builder(buffer)
.setByteBufferProvier(NativeClearedByteBufferProvider.INSTANCE);
columnBuilder.setComplexColumnSupplier(builder::build);
}
@Override
public ObjectStrategy<SerializablePairLongDouble> getObjectStrategy()
{
return new ObjectStrategy<SerializablePairLongDouble>()
{
@Override
public int compare(SerializablePairLongDouble o1, SerializablePairLongDouble o2)
{
return COMPARATOR.compare(o1, o2);
}
@Override
public Class<? extends SerializablePairLongDouble> getClazz()
{
return SerializablePairLongDouble.class;
}
@Override
public SerializablePairLongDouble fromByteBuffer(ByteBuffer buffer, int numBytes)
{
ByteBuffer readOnlyByteBuffer = buffer.asReadOnlyBuffer().order(buffer.order());
readOnlyByteBuffer.limit(buffer.position() + numBytes);
return SERDE.deserialize(readOnlyByteBuffer);
}
@Override
public byte[] toBytes(@Nullable SerializablePairLongDouble inPair)
{
return SERDE.serialize(inPair);
}
};
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongDoubleDeltaEncodedStagedSerde extends AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongDouble>
{
public SerializablePairLongDoubleDeltaEncodedStagedSerde(long minValue, boolean useIntegerDelta)
{
super(minValue, useIntegerDelta, SerializablePairLongDouble.class);
}
@Nullable
@Override
public SerializablePairLongDouble deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs;
if (useIntegerDelta) {
lhs = readOnlyBuffer.getInt();
} else {
lhs = readOnlyBuffer.getLong();
}
lhs += minValue;
Double rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getDouble();
}
return new SerializablePairLongDouble(lhs, rhs);
}
}

View File

@ -0,0 +1,53 @@
/*
* 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;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongDoubleSimpleStagedSerde extends AbstractSerializablePairLongObjectSimpleStagedSerde<SerializablePairLongDouble>
{
public SerializablePairLongDoubleSimpleStagedSerde()
{
super(SerializablePairLongDouble.class);
}
@Nullable
@Override
public SerializablePairLongDouble deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs = readOnlyBuffer.getLong();
Double rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getDouble();
}
return new SerializablePairLongDouble(lhs, rhs);
}
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.collections.SerializablePair;
import javax.annotation.Nullable;
public class SerializablePairLongFloat extends SerializablePair<Long, Float>
{
@JsonCreator
public SerializablePairLongFloat(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") @Nullable Float rhs)
{
super(lhs, rhs);
}
}

View File

@ -0,0 +1,66 @@
/*
* 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;
import javax.annotation.Nonnull;
public class SerializablePairLongFloatBufferStore extends AbstractSerializablePairLongObjectBufferStore<SerializablePairLongFloat>
{
public SerializablePairLongFloatBufferStore(SerializedStorage<SerializablePairLongFloat> serializedStorage)
{
super(serializedStorage);
}
@Override
@Nonnull
public AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongFloat> createColumnHeader()
{
long maxDelta = maxValue - minValue;
SerializablePairLongFloatColumnHeader columnHeader;
if (minValue < maxValue && maxDelta < 0 || minValue > maxValue) {
maxDelta = Long.MAX_VALUE;
minValue = 0;
}
if (maxDelta <= Integer.MAX_VALUE) {
columnHeader = new SerializablePairLongFloatColumnHeader(
SerializablePairLongFloatComplexMetricSerde.EXPECTED_VERSION,
true,
minValue
);
} else {
columnHeader = new SerializablePairLongFloatColumnHeader(
SerializablePairLongFloatComplexMetricSerde.EXPECTED_VERSION,
false,
minValue
);
}
return columnHeader;
}
@Override
public AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongFloat> createDeltaEncodedSerde(AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongFloat> columnHeader)
{
return new SerializablePairLongFloatDeltaEncodedStagedSerde(minValue, columnHeader.isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,39 @@
/*
* 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;
public class SerializablePairLongFloatColumnHeader extends AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongFloat>
{
SerializablePairLongFloatColumnHeader(byte[] bytes, long minValue)
{
super(bytes, minValue);
}
SerializablePairLongFloatColumnHeader(int version, boolean useIntegerDeltas, long minTimestamp)
{
super(version, useIntegerDeltas, minTimestamp);
}
@Override
public SerializablePairLongFloatDeltaEncodedStagedSerde createSerde()
{
return new SerializablePairLongFloatDeltaEncodedStagedSerde(minValue, isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,51 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
public class SerializablePairLongFloatColumnSerializer extends AbstractSerializablePairLongObjectColumnSerializer<SerializablePairLongFloat>
{
public SerializablePairLongFloatColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
ByteBufferProvider byteBufferProvider
)
{
super(new SerializablePairLongFloatSimpleStagedSerde(), segmentWriteOutMedium, byteBufferProvider);
}
@Override
public void open() throws IOException
{
Preconditions.checkState(state == State.START || state == State.OPEN, "open called in invalid state %s", state);
if (state == State.START) {
bufferStore = new SerializablePairLongFloatBufferStore(
new SerializedStorage<>(segmentWriteOutMedium.makeWriteOutBytes(), stagedSerde)
);
state = State.OPEN;
}
}
}

View File

@ -0,0 +1,129 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.CellReader;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongFloatComplexColumn implements ComplexColumn
{
private final Closer closer;
private final int serializedSize;
private final CellReader cellReader;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
public SerializablePairLongFloatComplexColumn(
CellReader cellReader,
AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde,
Closer closer,
int serializedSize
)
{
this.cellReader = cellReader;
this.serde = serde;
this.closer = closer;
this.serializedSize = serializedSize;
}
@Override
public Class<?> getClazz()
{
return SerializablePairLongFloat.class;
}
@Override
public String getTypeName()
{
return SerializablePairLongFloatComplexMetricSerde.TYPE_NAME;
}
@Override
public Object getRowValue(int rowNum)
{
return serde.deserialize(cellReader.getCell(rowNum));
}
@Override
public int getLength()
{
return serializedSize;
}
@Override
public void close()
{
try {
closer.close();
}
catch (IOException e) {
throw new RE(e, "error closing " + getClass().getName());
}
}
public static class Builder
{
private final int serializedSize;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
private final CellReader.Builder cellReaderBuilder;
public Builder(ByteBuffer buffer)
{
ByteBuffer masterByteBuffer = buffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
serializedSize = masterByteBuffer.remaining();
AbstractSerializablePairLongObjectColumnHeader<?> columnHeader =
AbstractSerializablePairLongObjectColumnHeader.fromBuffer(masterByteBuffer, SerializablePairLongFloat.class);
Preconditions.checkArgument(
columnHeader.getVersion() == SerializablePairLongFloatComplexMetricSerde.EXPECTED_VERSION,
"version %s expected, got %s",
SerializablePairLongFloatComplexMetricSerde.EXPECTED_VERSION,
columnHeader.getVersion()
);
serde = columnHeader.createSerde();
cellReaderBuilder = new CellReader.Builder(masterByteBuffer);
}
public Builder setByteBufferProvier(ByteBufferProvider byteBufferProvider)
{
cellReaderBuilder.setByteBufferProvider(byteBufferProvider);
return this;
}
public SerializablePairLongFloatComplexColumn build()
{
Closer closer = Closer.create();
CellReader cellReader = cellReaderBuilder.build();
closer.register(cellReader);
return new SerializablePairLongFloatComplexColumn(cellReader, serde, closer, serializedSize);
}
}
}

View File

@ -0,0 +1,110 @@
/*
* 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;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Comparator;
public class SerializablePairLongFloatComplexMetricSerde extends AbstractSerializableLongObjectPairSerde<SerializablePairLongFloat>
{
public static final int EXPECTED_VERSION = 3;
public static final String TYPE_NAME = "serializablePairLongFloat";
private static final SerializablePairLongFloatSimpleStagedSerde SERDE = new SerializablePairLongFloatSimpleStagedSerde();
private static final Comparator<SerializablePair<Long, Float>> COMPARATOR = SerializablePair.createNullHandlingComparator(
Float::compare,
true
);
public SerializablePairLongFloatComplexMetricSerde()
{
super(SerializablePairLongFloat.class);
}
@Override
public String getTypeName()
{
return TYPE_NAME;
}
@Override
public GenericColumnSerializer<SerializablePairLongFloat> getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return new SerializablePairLongFloatColumnSerializer(
segmentWriteOutMedium,
NativeClearedByteBufferProvider.INSTANCE
);
}
@Override
public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
{
SerializablePairLongFloatComplexColumn.Builder builder =
new SerializablePairLongFloatComplexColumn.Builder(buffer)
.setByteBufferProvier(NativeClearedByteBufferProvider.INSTANCE);
columnBuilder.setComplexColumnSupplier(builder::build);
}
@Override
public ObjectStrategy<SerializablePairLongFloat> getObjectStrategy()
{
return new ObjectStrategy<SerializablePairLongFloat>()
{
@Override
public int compare(SerializablePairLongFloat o1, SerializablePairLongFloat o2)
{
return COMPARATOR.compare(o1, o2);
}
@Override
public Class<? extends SerializablePairLongFloat> getClazz()
{
return SerializablePairLongFloat.class;
}
@Override
public SerializablePairLongFloat fromByteBuffer(ByteBuffer buffer, int numBytes)
{
ByteBuffer readOnlyByteBuffer = buffer.asReadOnlyBuffer().order(buffer.order());
readOnlyByteBuffer.limit(buffer.position() + numBytes);
return SERDE.deserialize(readOnlyByteBuffer);
}
@Override
public byte[] toBytes(@Nullable SerializablePairLongFloat inPair)
{
return SERDE.serialize(inPair);
}
};
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongFloatDeltaEncodedStagedSerde extends AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongFloat>
{
public SerializablePairLongFloatDeltaEncodedStagedSerde(long minValue, boolean useIntegerDelta)
{
super(minValue, useIntegerDelta, SerializablePairLongFloat.class);
}
@Nullable
@Override
public SerializablePairLongFloat deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs;
if (useIntegerDelta) {
lhs = readOnlyBuffer.getInt();
} else {
lhs = readOnlyBuffer.getLong();
}
lhs += minValue;
Float rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getFloat();
}
return new SerializablePairLongFloat(lhs, rhs);
}
}

View File

@ -0,0 +1,53 @@
/*
* 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;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongFloatSimpleStagedSerde extends AbstractSerializablePairLongObjectSimpleStagedSerde<SerializablePairLongFloat>
{
public SerializablePairLongFloatSimpleStagedSerde()
{
super(SerializablePairLongFloat.class);
}
@Nullable
@Override
public SerializablePairLongFloat deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs = readOnlyBuffer.getLong();
Float rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getFloat();
}
return new SerializablePairLongFloat(lhs, rhs);
}
}

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.collections.SerializablePair;
import javax.annotation.Nullable;
public class SerializablePairLongLong extends SerializablePair<Long, Long>
{
@JsonCreator
public SerializablePairLongLong(@JsonProperty("lhs") Long lhs, @JsonProperty("rhs") @Nullable Long rhs)
{
super(lhs, rhs);
}
}

View File

@ -0,0 +1,65 @@
/*
* 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;
import javax.annotation.Nonnull;
public class SerializablePairLongLongBufferStore extends AbstractSerializablePairLongObjectBufferStore<SerializablePairLongLong>
{
public SerializablePairLongLongBufferStore(SerializedStorage<SerializablePairLongLong> serializedStorage)
{
super(serializedStorage);
}
@Override
@Nonnull
public AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongLong> createColumnHeader()
{
long maxDelta = maxValue - minValue;
SerializablePairLongLongColumnHeader columnHeader;
if (minValue < maxValue && maxDelta < 0 || minValue > maxValue) {
maxDelta = Long.MAX_VALUE;
minValue = 0;
}
if (maxDelta <= Integer.MAX_VALUE) {
columnHeader = new SerializablePairLongLongColumnHeader(
SerializablePairLongLongComplexMetricSerde.EXPECTED_VERSION,
true,
minValue
);
} else {
columnHeader = new SerializablePairLongLongColumnHeader(
SerializablePairLongLongComplexMetricSerde.EXPECTED_VERSION,
false,
minValue
);
}
return columnHeader;
}
@Override
public AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongLong> createDeltaEncodedSerde(AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongLong> columnHeader)
{
return new SerializablePairLongLongDeltaEncodedStagedSerde(minValue, columnHeader.isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,39 @@
/*
* 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;
public class SerializablePairLongLongColumnHeader extends AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongLong>
{
SerializablePairLongLongColumnHeader(byte[] bytes, long minValue)
{
super(bytes, minValue);
}
SerializablePairLongLongColumnHeader(int version, boolean useIntegerDeltas, long minTimestamp)
{
super(version, useIntegerDeltas, minTimestamp);
}
@Override
public SerializablePairLongLongDeltaEncodedStagedSerde createSerde()
{
return new SerializablePairLongLongDeltaEncodedStagedSerde(minValue, isUseIntegerDeltas());
}
}

View File

@ -0,0 +1,51 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
public class SerializablePairLongLongColumnSerializer extends AbstractSerializablePairLongObjectColumnSerializer<SerializablePairLongLong>
{
public SerializablePairLongLongColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
ByteBufferProvider byteBufferProvider
)
{
super(new SerializablePairLongLongSimpleStagedSerde(), segmentWriteOutMedium, byteBufferProvider);
}
@Override
public void open() throws IOException
{
Preconditions.checkState(state == State.START || state == State.OPEN, "open called in invalid state %s", state);
if (state == State.START) {
bufferStore = new SerializablePairLongLongBufferStore(
new SerializedStorage<>(segmentWriteOutMedium.makeWriteOutBytes(), stagedSerde)
);
state = State.OPEN;
}
}
}

View File

@ -0,0 +1,129 @@
/*
* 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;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.CellReader;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongLongComplexColumn implements ComplexColumn
{
private final Closer closer;
private final int serializedSize;
private final CellReader cellReader;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
public SerializablePairLongLongComplexColumn(
CellReader cellReader,
AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde,
Closer closer,
int serializedSize
)
{
this.cellReader = cellReader;
this.serde = serde;
this.closer = closer;
this.serializedSize = serializedSize;
}
@Override
public Class<?> getClazz()
{
return SerializablePairLongLong.class;
}
@Override
public String getTypeName()
{
return SerializablePairLongLongComplexMetricSerde.TYPE_NAME;
}
@Override
public Object getRowValue(int rowNum)
{
return serde.deserialize(cellReader.getCell(rowNum));
}
@Override
public int getLength()
{
return serializedSize;
}
@Override
public void close()
{
try {
closer.close();
}
catch (IOException e) {
throw new RE(e, "error closing " + getClass().getName());
}
}
public static class Builder
{
private final int serializedSize;
private final AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<?> serde;
private final CellReader.Builder cellReaderBuilder;
public Builder(ByteBuffer buffer)
{
ByteBuffer masterByteBuffer = buffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
serializedSize = masterByteBuffer.remaining();
AbstractSerializablePairLongObjectColumnHeader<?> columnHeader =
AbstractSerializablePairLongObjectColumnHeader.fromBuffer(masterByteBuffer, SerializablePairLongLong.class);
Preconditions.checkArgument(
columnHeader.getVersion() == SerializablePairLongLongComplexMetricSerde.EXPECTED_VERSION,
"version %s expected, got %s",
SerializablePairLongLongComplexMetricSerde.EXPECTED_VERSION,
columnHeader.getVersion()
);
serde = columnHeader.createSerde();
cellReaderBuilder = new CellReader.Builder(masterByteBuffer);
}
public Builder setByteBufferProvier(ByteBufferProvider byteBufferProvider)
{
cellReaderBuilder.setByteBufferProvider(byteBufferProvider);
return this;
}
public SerializablePairLongLongComplexColumn build()
{
Closer closer = Closer.create();
CellReader cellReader = cellReaderBuilder.build();
closer.register(cellReader);
return new SerializablePairLongLongComplexColumn(cellReader, serde, closer, serializedSize);
}
}
}

View File

@ -0,0 +1,109 @@
/*
* 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;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.data.ObjectStrategy;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Comparator;
public class SerializablePairLongLongComplexMetricSerde extends AbstractSerializableLongObjectPairSerde<SerializablePairLongLong>
{
public static final int EXPECTED_VERSION = 3;
public static final String TYPE_NAME = "serializablePairLongLong";
private static final SerializablePairLongLongSimpleStagedSerde SERDE = new SerializablePairLongLongSimpleStagedSerde();
private static final Comparator<SerializablePair<Long, Long>> COMPARATOR = SerializablePair.createNullHandlingComparator(
Long::compare,
true
);
public SerializablePairLongLongComplexMetricSerde()
{
super(SerializablePairLongLong.class);
}
@Override
public String getTypeName()
{
return TYPE_NAME;
}
@Override
public GenericColumnSerializer<SerializablePairLongLong> getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return new SerializablePairLongLongColumnSerializer(
segmentWriteOutMedium,
NativeClearedByteBufferProvider.INSTANCE
);
}
@Override
public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
{
SerializablePairLongLongComplexColumn.Builder builder =
new SerializablePairLongLongComplexColumn.Builder(buffer)
.setByteBufferProvier(NativeClearedByteBufferProvider.INSTANCE);
columnBuilder.setComplexColumnSupplier(builder::build);
}
@Override
public ObjectStrategy<SerializablePairLongLong> getObjectStrategy()
{
return new ObjectStrategy<SerializablePairLongLong>()
{
@Override
public int compare(SerializablePairLongLong o1, SerializablePairLongLong o2)
{
return COMPARATOR.compare(o1, o2);
}
@Override
public Class<? extends SerializablePairLongLong> getClazz()
{
return SerializablePairLongLong.class;
}
@Override
public SerializablePairLongLong fromByteBuffer(ByteBuffer buffer, int numBytes)
{
ByteBuffer readOnlyByteBuffer = buffer.asReadOnlyBuffer().order(buffer.order());
readOnlyByteBuffer.limit(buffer.position() + numBytes);
return SERDE.deserialize(readOnlyByteBuffer);
}
@Override
public byte[] toBytes(@Nullable SerializablePairLongLong inPair)
{
return SERDE.serialize(inPair);
}
};
}
}

View File

@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongLongDeltaEncodedStagedSerde extends AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongLong>
{
public SerializablePairLongLongDeltaEncodedStagedSerde(long minValue, boolean useIntegerDelta)
{
super(minValue, useIntegerDelta, SerializablePairLongLong.class);
}
@Nullable
@Override
public SerializablePairLongLong deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs;
if (useIntegerDelta) {
lhs = readOnlyBuffer.getInt();
} else {
lhs = readOnlyBuffer.getLong();
}
lhs += minValue;
Long rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getLong();
}
return new SerializablePairLongLong(lhs, rhs);
}
}

View File

@ -0,0 +1,53 @@
/*
* 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;
import org.apache.druid.common.config.NullHandling;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
public class SerializablePairLongLongSimpleStagedSerde extends AbstractSerializablePairLongObjectSimpleStagedSerde<SerializablePairLongLong>
{
public SerializablePairLongLongSimpleStagedSerde()
{
super(SerializablePairLongLong.class);
}
@Nullable
@Override
public SerializablePairLongLong deserialize(ByteBuffer byteBuffer)
{
if (byteBuffer.remaining() == 0) {
return null;
}
ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder());
long lhs = readOnlyBuffer.getLong();
Long rhs = null;
if (readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE) {
rhs = readOnlyBuffer.getLong();
}
return new SerializablePairLongLong(lhs, rhs);
}
}

View File

@ -19,93 +19,23 @@
package org.apache.druid.query.aggregation;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.segment.serde.Serializer;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.CellWriter;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
public class SerializablePairLongStringBufferStore
public class SerializablePairLongStringBufferStore extends AbstractSerializablePairLongObjectBufferStore<SerializablePairLongString>
{
private final SerializedStorage<SerializablePairLongString> serializedStorage;
private long minValue = Long.MAX_VALUE;
private long maxValue = Long.MIN_VALUE;
public SerializablePairLongStringBufferStore(SerializedStorage<SerializablePairLongString> serializedStorage)
{
this.serializedStorage = serializedStorage;
}
public void store(@Nullable SerializablePairLongString pairLongString) throws IOException
{
if (pairLongString != null && pairLongString.lhs != null) {
minValue = Math.min(minValue, pairLongString.lhs);
maxValue = Math.max(maxValue, pairLongString.lhs);
}
serializedStorage.store(pairLongString);
}
/**
* each call transfers the temporary buffer into an encoded, block-compessed buffer of the segment. It is ready to be
* transferred to a {@link WritableByteChannel}
*
* @param byteBufferProvider - provides a ByteBuffer used for block compressed encoding
* @param segmentWriteOutMedium - used to create temporary storage
* @return encoded buffer ready to be stored
* @throws IOException
*/
public TransferredBuffer transferToRowWriter(
ByteBufferProvider byteBufferProvider,
SegmentWriteOutMedium segmentWriteOutMedium
) throws IOException
{
SerializablePairLongStringColumnHeader columnHeader = createColumnHeader();
SerializablePairLongStringDeltaEncodedStagedSerde serde =
new SerializablePairLongStringDeltaEncodedStagedSerde(
columnHeader.getMinValue(),
columnHeader.isUseIntegerDeltas()
);
// try-with-resources will call cellWriter.close() an extra time in the normal case, but it protects against
// buffer leaking in the case of an exception (close() is idempotent). In the normal path, close() performs some
// finalization of the CellWriter object. We want that object state finalized before creating the TransferredBuffer
// as a point of good style (though strictly speaking, it works fine to pass it in before calling close since
// TransferredBuffer does not do anything in the constructor with the object)
try (CellWriter cellWriter =
new CellWriter.Builder(segmentWriteOutMedium).setByteBufferProvider(byteBufferProvider).build()) {
try (IOIterator<SerializablePairLongString> bufferIterator = iterator()) {
while (bufferIterator.hasNext()) {
SerializablePairLongString pairLongString = bufferIterator.next();
byte[] serialized = serde.serialize(pairLongString);
cellWriter.write(serialized);
}
cellWriter.close();
return new TransferredBuffer(cellWriter, columnHeader);
}
}
super(serializedStorage);
}
@Override
@Nonnull
public SerializablePairLongStringColumnHeader createColumnHeader()
public AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongString> createColumnHeader()
{
long maxDelta = maxValue - minValue;
SerializablePairLongStringColumnHeader columnHeader;
if (minValue < maxValue && maxDelta < 0 || minValue > maxValue) {
// true iff
// 1. we have overflow in our range || 2. we have only seen null values
// in this case, effectively disable delta encoding by using longs and a min value of 0
maxDelta = Long.MAX_VALUE;
minValue = 0;
}
@ -126,37 +56,9 @@ public class SerializablePairLongStringBufferStore
return columnHeader;
}
public IOIterator<SerializablePairLongString> iterator() throws IOException
@Override
public AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongString> createDeltaEncodedSerde(AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongString> columnHeader)
{
return serializedStorage.iterator();
}
/**
* contains serialized data that is compressed and delta-encoded (Long)
* It's ready to be transferred to a {@link WritableByteChannel}
*/
public static class TransferredBuffer implements Serializer
{
private final CellWriter cellWriter;
private final SerializablePairLongStringColumnHeader columnHeader;
public TransferredBuffer(CellWriter cellWriter, SerializablePairLongStringColumnHeader columnHeader)
{
this.cellWriter = cellWriter;
this.columnHeader = columnHeader;
}
@Override
public void writeTo(WritableByteChannel channel, @Nullable FileSmoosher smoosher) throws IOException
{
columnHeader.transferTo(channel);
cellWriter.writeTo(channel, smoosher);
}
@Override
public long getSerializedSize()
{
return columnHeader.getSerializedSize() + cellWriter.getSerializedSize();
}
return new SerializablePairLongStringDeltaEncodedStagedSerde(minValue, columnHeader.isUseIntegerDeltas());
}
}

View File

@ -19,93 +19,21 @@
package org.apache.druid.query.aggregation;
import com.google.common.base.MoreObjects;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.serde.cell.LongSerializer;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
public class SerializablePairLongStringColumnHeader
public class SerializablePairLongStringColumnHeader extends AbstractSerializablePairLongObjectColumnHeader<SerializablePairLongString>
{
// header size is 4 bytes for word alignment for LZ4 (minmatch) compression
private static final int HEADER_SIZE_BYTES = 4;
private static final int USE_INTEGER_MASK = 0x80;
private static final int VERSION_INDEX = 0;
private static final int ENCODING_INDEX = 1;
private final byte[] bytes;
private final long minValue;
private SerializablePairLongStringColumnHeader(byte[] bytes, long minTimestamp)
SerializablePairLongStringColumnHeader(byte[] bytes, long minTimestamp)
{
this.bytes = bytes;
this.minValue = minTimestamp;
super(bytes, minTimestamp);
}
public SerializablePairLongStringColumnHeader(int version, boolean useIntegerDeltas, long minTimestamp)
{
this.minValue = minTimestamp;
bytes = new byte[HEADER_SIZE_BYTES];
Preconditions.checkArgument(version <= 255, "max version 255");
bytes[VERSION_INDEX] = (byte) version;
if (useIntegerDeltas) {
bytes[ENCODING_INDEX] |= USE_INTEGER_MASK;
}
}
public static SerializablePairLongStringColumnHeader fromBuffer(ByteBuffer byteBuffer)
{
byte[] bytes = new byte[HEADER_SIZE_BYTES];
byteBuffer.get(bytes);
long minTimestamp = byteBuffer.getLong();
return new SerializablePairLongStringColumnHeader(bytes, minTimestamp);
super(version, useIntegerDeltas, minTimestamp);
}
@Override
public SerializablePairLongStringDeltaEncodedStagedSerde createSerde()
{
return new SerializablePairLongStringDeltaEncodedStagedSerde(minValue, isUseIntegerDeltas());
}
public void transferTo(WritableByteChannel channel) throws IOException
{
LongSerializer longSerializer = new LongSerializer();
channel.write(ByteBuffer.wrap(bytes));
channel.write(longSerializer.serialize(minValue));
}
public int getVersion()
{
return 0XFF & bytes[VERSION_INDEX];
}
public boolean isUseIntegerDeltas()
{
return (bytes[ENCODING_INDEX] & USE_INTEGER_MASK) != 0;
}
public long getMinValue()
{
return minValue;
}
public int getSerializedSize()
{
return HEADER_SIZE_BYTES + Long.BYTES;
}
@Override
public String toString()
{
return MoreObjects.toStringHelper(this)
.add("bytes", bytes)
.add("minValue", minValue)
.toString();
}
}

View File

@ -20,16 +20,10 @@
package org.apache.druid.query.aggregation;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.serde.cell.ByteBufferProvider;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
/**
* valid call sequence
@ -39,85 +33,27 @@ import java.nio.channels.WritableByteChannel;
* getSerializedSize() / writeTo() effectively function as a close call, but each may be called multiple times and has
* no effect on one another.
*/
@SuppressWarnings("NotNullFieldNotInitialized")
public class SerializablePairLongStringColumnSerializer implements GenericColumnSerializer<SerializablePairLongString>
public class SerializablePairLongStringColumnSerializer extends AbstractSerializablePairLongObjectColumnSerializer<SerializablePairLongString>
{
public static final StagedSerde<SerializablePairLongString> STAGED_SERDE =
new SerializablePairLongStringSimpleStagedSerde();
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final ByteBufferProvider byteBufferProvider;
private State state = State.START;
private SerializablePairLongStringBufferStore bufferStore;
private SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer;
public SerializablePairLongStringColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
ByteBufferProvider byteBufferProvider
)
{
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.byteBufferProvider = byteBufferProvider;
super(new SerializablePairLongStringSimpleStagedSerde(), segmentWriteOutMedium, byteBufferProvider);
}
@Override
public void open() throws IOException
{
Preconditions.checkState(state == State.START || state == State.OPEN, "open called in invalid state %s", state);
Preconditions.checkState(state == AbstractSerializablePairLongObjectColumnSerializer.State.START || state == AbstractSerializablePairLongObjectColumnSerializer.State.OPEN, "open called in invalid state %s", state);
if (state == State.START) {
if (state == AbstractSerializablePairLongObjectColumnSerializer.State.START) {
bufferStore = new SerializablePairLongStringBufferStore(
new SerializedStorage<>(segmentWriteOutMedium.makeWriteOutBytes(), STAGED_SERDE)
new SerializedStorage<>(segmentWriteOutMedium.makeWriteOutBytes(), stagedSerde)
);
state = State.OPEN;
state = AbstractSerializablePairLongObjectColumnSerializer.State.OPEN;
}
}
@Override
public void serialize(ColumnValueSelector<? extends SerializablePairLongString> selector) throws IOException
{
Preconditions.checkState(state == State.OPEN, "serialize called in invalid state %s", state);
SerializablePairLongString pairLongString = selector.getObject();
bufferStore.store(pairLongString);
}
@Override
public long getSerializedSize() throws IOException
{
Preconditions.checkState(
state != State.START,
"getSerializedSize called in invalid state %s (must have opened at least)",
state
);
transferToRowWriterIfNecessary();
return transferredBuffer.getSerializedSize();
}
@Override
public void writeTo(WritableByteChannel channel, @Nullable FileSmoosher smoosher) throws IOException
{
Preconditions.checkState(state != State.START, "writeTo called in invalid state %s", state);
transferToRowWriterIfNecessary();
transferredBuffer.writeTo(channel, smoosher);
}
private void transferToRowWriterIfNecessary() throws IOException
{
if (state == State.OPEN) {
transferredBuffer = bufferStore.transferToRowWriter(byteBufferProvider, segmentWriteOutMedium);
state = State.CLOSED;
}
}
private enum State
{
START,
OPEN,
CLOSED,
}
}

View File

@ -101,7 +101,7 @@ public class SerializablePairLongStringComplexColumn implements ComplexColumn
serializedSize = masterByteBuffer.remaining();
SerializablePairLongStringColumnHeader columnHeader =
SerializablePairLongStringColumnHeader.fromBuffer(masterByteBuffer);
(SerializablePairLongStringColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(masterByteBuffer, SerializablePairLongString.class);
Preconditions.checkArgument(
columnHeader.getVersion() == SerializablePairLongStringComplexMetricSerde.EXPECTED_VERSION,

View File

@ -22,7 +22,6 @@ package org.apache.druid.query.aggregation;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.serde.cell.StorableBuffer;
import javax.annotation.Nullable;
@ -37,15 +36,11 @@ import java.nio.ByteOrder;
* otherwise
* Long:Integer:bytes
*/
public class SerializablePairLongStringDeltaEncodedStagedSerde implements StagedSerde<SerializablePairLongString>
public class SerializablePairLongStringDeltaEncodedStagedSerde extends AbstractSerializablePairLongObjectDeltaEncodedStagedSerde<SerializablePairLongString>
{
private final long minValue;
private final boolean useIntegerDelta;
public SerializablePairLongStringDeltaEncodedStagedSerde(long minValue, boolean useIntegerDelta)
{
this.minValue = minValue;
this.useIntegerDelta = useIntegerDelta;
super(minValue, useIntegerDelta, SerializablePairLongString.class);
}
@Override

View File

@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.serde.cell.StagedSerde;
import org.apache.druid.segment.serde.cell.StorableBuffer;
import javax.annotation.Nullable;
@ -35,8 +34,14 @@ import java.nio.ByteOrder;
* or
* Long:StringSize:StringData
*/
public class SerializablePairLongStringSimpleStagedSerde implements StagedSerde<SerializablePairLongString>
public class SerializablePairLongStringSimpleStagedSerde extends AbstractSerializablePairLongObjectSimpleStagedSerde<SerializablePairLongString>
{
public SerializablePairLongStringSimpleStagedSerde()
{
super(SerializablePairLongString.class);
}
@Override
public StorableBuffer serializeDelayed(@Nullable SerializablePairLongString value)
{

View File

@ -19,7 +19,6 @@
package org.apache.druid.query.aggregation.any;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.VectorAggregator;
@ -43,10 +42,6 @@ public class NilVectorAggregator implements VectorAggregator
NullHandling.defaultLongValue()
);
public static final SerializablePair<Long, Double> DOUBLE_NIL_PAIR = new SerializablePair<>(0L, NullHandling.defaultDoubleValue());
public static final SerializablePair<Long, Long> LONG_NIL_PAIR = new SerializablePair<>(0L, NullHandling.defaultLongValue());
public static final SerializablePair<Long, Float> FLOAT_NIL_PAIR = new SerializablePair<>(0L, NullHandling.defaultFloatValue());
/**
* @return A vectorized aggregator that returns the default double value.
*/
@ -74,11 +69,6 @@ public class NilVectorAggregator implements VectorAggregator
@Nullable
private final Object returnValue;
public static NilVectorAggregator of(Object returnValue)
{
return new NilVectorAggregator(returnValue);
}
private NilVectorAggregator(@Nullable Object returnValue)
{
this.returnValue = returnValue;

View File

@ -19,30 +19,36 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class DoubleFirstAggregator extends NumericFirstAggregator<BaseDoubleColumnValueSelector>
public class DoubleFirstAggregator extends NumericFirstAggregator
{
double firstValue;
public DoubleFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector)
public DoubleFirstAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
firstValue = 0;
}
@Override
void setCurrentValue()
void setFirstValue()
{
firstValue = valueSelector.getDouble();
}
@Override
void setFirstValue(Number firstValue)
{
this.firstValue = firstValue.doubleValue();
}
@Override
public Object get()
{
return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
return new SerializablePairLongDouble(firstTime, rhsNull ? null : firstValue);
}
@Override

View File

@ -21,19 +21,18 @@ package org.apache.druid.query.aggregation.first;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.query.aggregation.SerializablePairLongDoubleComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -43,6 +42,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -53,11 +53,15 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
@JsonTypeName("doubleFirst")
public class DoubleFirstAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongDoubleComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new DoubleFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -69,7 +73,9 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new DoubleFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -112,13 +118,18 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new DoubleFirstAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongDouble.class
)
);
}
}
@ -126,13 +137,18 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new DoubleFirstBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongDouble.class
)
);
}
}
@ -142,14 +158,15 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
VectorColumnSelectorFactory columnSelectorFactory
)
{
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(
timeColumn);
return new DoubleFirstVectorAggregator(timeSelector, valueSelector);
}
return NilVectorAggregator.of(NilVectorAggregator.DOUBLE_NIL_PAIR);
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new DoubleFirstVectorAggregator(timeSelector, objectSelector);
}
@Override
@ -180,74 +197,13 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("DoubleFirstAggregatorFactory is not supported during ingestion for rollup");
return new GenericFirstAggregateCombiner(SerializablePairLongDouble.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new DoubleFirstAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleFirstAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Double> pair = selector.getObject();
if (pair.lhs < firstTime) {
firstTime = pair.lhs;
if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Double> pair = (SerializablePair<Long, Double>) selector.getObject();
long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new DoubleFirstAggregatorFactory(name, name, timeColumn);
}
@Override
@ -255,16 +211,16 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongDouble(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
return new SerializablePairLongDouble(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Double>) object).rhs;
return object == null ? null : ((SerializablePairLongDouble) object).rhs;
}
@Override
@ -304,8 +260,7 @@ public class DoubleFirstAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE;
return TYPE;
}
@Override

View File

@ -19,20 +19,21 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class DoubleFirstBufferAggregator extends NumericFirstBufferAggregator<BaseDoubleColumnValueSelector>
public class DoubleFirstBufferAggregator extends NumericFirstBufferAggregator
{
public DoubleFirstBufferAggregator(
BaseLongColumnValueSelector timeSelector,
BaseDoubleColumnValueSelector valueSelector
ColumnValueSelector valueSelector,
boolean needsFoldCheck
)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -42,16 +43,22 @@ public class DoubleFirstBufferAggregator extends NumericFirstBufferAggregator<Ba
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSector)
{
buf.putDouble(position, valueSelector.getDouble());
buf.putDouble(position, valueSector.getDouble());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putDouble(position, value.doubleValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
return new SerializablePairLongDouble(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -28,9 +29,14 @@ import java.nio.ByteBuffer;
public class DoubleFirstVectorAggregator extends NumericFirstVectorAggregator
{
public DoubleFirstVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, null, objectSelector);
}
public DoubleFirstVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, null);
}
@Override
@ -43,7 +49,7 @@ public class DoubleFirstVectorAggregator extends NumericFirstVectorAggregator
@Override
void putValue(ByteBuffer buf, int position, int index)
{
double firstValue = valueSelector.getDoubleVector()[index];
double firstValue = valueSelector != null ? valueSelector.getDoubleVector()[index] : ((SerializablePairLongDouble) (objectSelector.getObjectVector()[index])).getRhs();
buf.putDouble(position, firstValue);
}
@ -56,6 +62,6 @@ public class DoubleFirstVectorAggregator extends NumericFirstVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
return new SerializablePairLongDouble(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation.first;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import javax.annotation.Nullable;
public class FirstLastUtils
{
/**
* Returns whether a given value selector *might* contain object assignable from pairClass (SerializablePairLong*).
*/
public static boolean selectorNeedsFoldCheck(
final BaseObjectColumnValueSelector<?> valueSelector,
@Nullable final ColumnCapabilities valueSelectorCapabilities,
Class pairClass
)
{
if (valueSelectorCapabilities != null && !valueSelectorCapabilities.is(ValueType.COMPLEX)) {
// Known, non-complex type.
return false;
}
if (valueSelector instanceof NilColumnValueSelector) {
// Nil column, definitely no SerializablePairLongObject.
return false;
}
// Check if the selector class could possibly be of pairClass* (either a superclass or subclass).
final Class<?> clazz = valueSelector.classOfObject();
return clazz.isAssignableFrom(pairClass)
|| pairClass.isAssignableFrom(clazz);
}
/**
* Returns whether an object *might* is assignable to/from the pairClass.
*/
public static boolean objectNeedsFoldCheck(Object obj, Class pairClass)
{
if (obj == null) {
return false;
}
final Class<?> clazz = obj.getClass();
return clazz.isAssignableFrom(pairClass)
|| pairClass.isAssignableFrom(clazz);
}
}

View File

@ -19,33 +19,40 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class FloatFirstAggregator extends NumericFirstAggregator<BaseFloatColumnValueSelector>
public class FloatFirstAggregator extends NumericFirstAggregator
{
float firstValue;
public FloatFirstAggregator(
BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector
ColumnValueSelector valueSelector,
boolean needsFoldCheck
)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
firstValue = 0;
}
@Override
void setCurrentValue()
void setFirstValue()
{
firstValue = valueSelector.getFloat();
}
@Override
void setFirstValue(Number firstValue)
{
this.firstValue = firstValue.floatValue();
}
@Override
public Object get()
{
return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
return new SerializablePairLongFloat(firstTime, rhsNull ? null : firstValue);
}
@Override

View File

@ -21,19 +21,18 @@ package org.apache.druid.query.aggregation.first;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.query.aggregation.SerializablePairLongFloatComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -43,6 +42,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -53,11 +53,15 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
@JsonTypeName("floatFirst")
public class FloatFirstAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongFloatComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new FloatFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -69,7 +73,8 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new FloatFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -104,13 +109,18 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new FloatFirstAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongFloat.class
)
);
}
}
@ -118,13 +128,18 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new FloatFirstBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongFloat.class
)
);
}
}
@ -132,13 +147,14 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
@Override
public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
{
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
return new FloatFirstVectorAggregator(timeSelector, valueSelector);
}
return NilVectorAggregator.of(NilVectorAggregator.FLOAT_NIL_PAIR);
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new FloatFirstVectorAggregator(timeSelector, objectSelector);
}
@Override
@ -175,73 +191,13 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("FloatFirstAggregatorFactory is not supported during ingestion for rollup");
return new GenericFirstAggregateCombiner(SerializablePairLongFloat.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new FloatFirstAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatFirstAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Float> pair = selector.getObject();
if (pair.lhs < firstTime) {
firstTime = pair.lhs;
if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Float> pair = selector.getObject();
long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new FloatFirstAggregatorFactory(name, name, timeColumn);
}
@ -250,16 +206,16 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongFloat(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
return new SerializablePairLongFloat(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Float>) object).rhs;
return object == null ? null : ((SerializablePairLongFloat) object).rhs;
}
@Override
@ -299,8 +255,7 @@ public class FloatFirstAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return ColumnType.FLOAT;
return TYPE;
}
@Override

View File

@ -19,20 +19,21 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class FloatFirstBufferAggregator extends NumericFirstBufferAggregator<BaseFloatColumnValueSelector>
public class FloatFirstBufferAggregator extends NumericFirstBufferAggregator
{
public FloatFirstBufferAggregator(
BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector
ColumnValueSelector valueSelector,
boolean needsFoldCheck
)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -42,16 +43,22 @@ public class FloatFirstBufferAggregator extends NumericFirstBufferAggregator<Bas
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSector)
{
buf.putFloat(position, valueSelector.getFloat());
buf.putFloat(position, valueSector.getFloat());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putFloat(position, value.floatValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
return new SerializablePairLongFloat(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -28,9 +29,14 @@ import java.nio.ByteBuffer;
public class FloatFirstVectorAggregator extends NumericFirstVectorAggregator
{
public FloatFirstVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, null, objectSelector);
}
public FloatFirstVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, null);
}
@Override
@ -39,11 +45,10 @@ public class FloatFirstVectorAggregator extends NumericFirstVectorAggregator
buf.putFloat(position, 0);
}
@Override
void putValue(ByteBuffer buf, int position, int index)
{
float firstValue = valueSelector.getFloatVector()[index];
float firstValue = valueSelector != null ? valueSelector.getFloatVector()[index] : ((SerializablePairLongFloat) objectSelector.getObjectVector()[index]).getRhs();
buf.putFloat(position, firstValue);
}
@ -56,6 +61,6 @@ public class FloatFirstVectorAggregator extends NumericFirstVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
return new SerializablePairLongFloat(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
}
}

View File

@ -19,41 +19,48 @@
package org.apache.druid.query.aggregation.first;
import com.google.common.primitives.Longs;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.ObjectAggregateCombiner;
import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
public class StringFirstAggregateCombiner extends ObjectAggregateCombiner<SerializablePairLongString>
public class GenericFirstAggregateCombiner<T extends SerializablePair<Long, ?>> extends ObjectAggregateCombiner<T>
{
private SerializablePairLongString firstValue;
private T firstValue;
private final Class<T> pairClass;
public GenericFirstAggregateCombiner(Class<T> pairClass)
{
this.pairClass = pairClass;
}
@Override
public void reset(ColumnValueSelector selector)
{
firstValue = (SerializablePairLongString) selector.getObject();
firstValue = (T) selector.getObject();
}
@Override
public void fold(ColumnValueSelector selector)
{
SerializablePairLongString newValue = (SerializablePairLongString) selector.getObject();
if (StringFirstAggregatorFactory.TIME_COMPARATOR.compare(firstValue, newValue) > 0) {
T newValue = (T) selector.getObject();
if (Longs.compare(firstValue.lhs, newValue.lhs) > 0) {
firstValue = newValue;
}
}
@Nullable
@Override
public SerializablePairLongString getObject()
public T getObject()
{
return firstValue;
}
@Override
public Class<SerializablePairLongString> classOfObject()
public Class<T> classOfObject()
{
return SerializablePairLongString.class;
return pairClass;
}
}

View File

@ -19,29 +19,36 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class LongFirstAggregator extends NumericFirstAggregator<BaseLongColumnValueSelector>
public class LongFirstAggregator extends NumericFirstAggregator
{
long firstValue;
public LongFirstAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
public LongFirstAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
firstValue = 0;
}
@Override
void setCurrentValue()
void setFirstValue()
{
firstValue = valueSelector.getLong();
}
@Override
void setFirstValue(Number firstValue)
{
this.firstValue = firstValue.longValue();
}
@Override
public Object get()
{
return new SerializablePair<>(firstTime, rhsNull ? null : firstValue);
return new SerializablePairLongLong(firstTime, rhsNull ? null : firstValue);
}
@Override

View File

@ -21,19 +21,18 @@ package org.apache.druid.query.aggregation.first;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.query.aggregation.SerializablePairLongLongComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -43,6 +42,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -52,11 +52,15 @@ import java.util.Comparator;
import java.util.List;
import java.util.Map;
@JsonTypeName("longFirst")
public class LongFirstAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongLongComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new LongFirstAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -68,7 +72,8 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new LongFirstBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -103,13 +108,18 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new LongFirstAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongLong.class
)
);
}
}
@ -117,13 +127,18 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new LongFirstBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongLong.class
)
);
}
}
@ -131,14 +146,14 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
@Override
public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
{
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(
timeColumn);
return new LongFirstVectorAggregator(timeSelector, valueSelector);
}
return NilVectorAggregator.of(NilVectorAggregator.LONG_NIL_PAIR);
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new LongFirstVectorAggregator(timeSelector, objectSelector);
}
@Override
@ -175,72 +190,13 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("LongFirstAggregatorFactory is not supported during ingestion for rollup");
return new GenericFirstAggregateCombiner(SerializablePairLongLong.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new LongFirstAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongFirstAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Long> pair = selector.getObject();
if (pair.lhs < firstTime) {
firstTime = pair.lhs;
if (pair.rhs != null) {
firstValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Long> pair = selector.getObject();
long firstTime = buf.getLong(position);
if (pair.lhs < firstTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new LongFirstAggregatorFactory(name, name, timeColumn);
}
@Override
@ -248,16 +204,16 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongLong(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
return new SerializablePairLongLong(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Long>) object).rhs;
return object == null ? null : ((SerializablePairLongLong) object).rhs;
}
@Override
@ -297,8 +253,7 @@ public class LongFirstAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return ColumnType.LONG;
return TYPE;
}
@Override

View File

@ -19,16 +19,17 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class LongFirstBufferAggregator extends NumericFirstBufferAggregator<BaseLongColumnValueSelector>
public class LongFirstBufferAggregator extends NumericFirstBufferAggregator
{
public LongFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
public LongFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -38,16 +39,22 @@ public class LongFirstBufferAggregator extends NumericFirstBufferAggregator<Base
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSelector)
{
buf.putLong(position, valueSelector.getLong());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putLong(position, value.longValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
return new SerializablePairLongLong(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -27,9 +28,14 @@ import java.nio.ByteBuffer;
public class LongFirstVectorAggregator extends NumericFirstVectorAggregator
{
public LongFirstVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, null, objectSelector);
}
public LongFirstVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, null);
}
@Override
@ -38,15 +44,13 @@ public class LongFirstVectorAggregator extends NumericFirstVectorAggregator
buf.putLong(position, 0);
}
@Override
void putValue(ByteBuffer buf, int position, int index)
{
long firstValue = valueSelector.getLongVector()[index];
long firstValue = valueSelector != null ? valueSelector.getLongVector()[index] : ((SerializablePairLongLong) objectSelector.getObjectVector()[index]).getRhs();
buf.putLong(position, firstValue);
}
/**
* @return The object as a pair with the position and the value stored at the position in the buffer.
*/
@ -55,6 +59,6 @@ public class LongFirstVectorAggregator extends NumericFirstVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
return new SerializablePairLongLong(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
}
}

View File

@ -19,28 +19,31 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
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;
import org.apache.druid.segment.ColumnValueSelector;
/**
* Base type for on heap 'first' aggregator for primitive numeric column selectors
*/
public abstract class NumericFirstAggregator<TSelector extends BaseNullableColumnValueSelector> implements Aggregator
public abstract class NumericFirstAggregator implements Aggregator
{
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
private final boolean needsFoldCheck;
final TSelector valueSelector;
final ColumnValueSelector valueSelector;
long firstTime;
boolean rhsNull;
public NumericFirstAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
public NumericFirstAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
this.needsFoldCheck = needsFoldCheck;
firstTime = Long.MAX_VALUE;
rhsNull = !useDefault;
@ -49,7 +52,12 @@ public abstract class NumericFirstAggregator<TSelector extends BaseNullableColum
/**
* Store the current primitive typed 'first' value
*/
abstract void setCurrentValue();
abstract void setFirstValue();
/**
* Store a non-null first value
*/
abstract void setFirstValue(Number firstValue);
@Override
public void aggregate()
@ -57,13 +65,33 @@ public abstract class NumericFirstAggregator<TSelector extends BaseNullableColum
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
final Object object = valueSelector.getObject();
if (object instanceof SerializablePair) {
SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) object;
if (inPair.lhs < firstTime) {
firstTime = inPair.lhs;
if (inPair.rhs == null) {
rhsNull = true;
} else {
rhsNull = false;
setFirstValue(inPair.rhs);
}
}
return;
}
}
long time = timeSelector.getLong();
if (time < firstTime) {
firstTime = time;
if (useDefault || !valueSelector.isNull()) {
setCurrentValue();
setFirstValue();
rhsNull = false;
} else {
setFirstValue(0);
rhsNull = true;
}
}

View File

@ -19,32 +19,34 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
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 org.apache.druid.segment.ColumnValueSelector;
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
public abstract class NumericFirstBufferAggregator 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;
private final boolean needsFoldCheck;
final TSelector valueSelector;
final ColumnValueSelector valueSelector;
public NumericFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
public NumericFirstBufferAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
this.needsFoldCheck = needsFoldCheck;
}
/**
@ -55,13 +57,22 @@ public abstract class NumericFirstBufferAggregator<TSelector extends BaseNullabl
/**
* Place the primitive value in the buffer at the position of {@link #VALUE_OFFSET}
*/
abstract void putValue(ByteBuffer buf, int position);
abstract void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSector);
void updateTimeWithValue(ByteBuffer buf, int position, long time)
abstract void putValue(ByteBuffer buf, int position, Number value);
void updateTimeWithValue(ByteBuffer buf, int position, long time, ColumnValueSelector valueSelector)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET);
putValue(buf, position + VALUE_OFFSET, valueSelector);
}
void updateTimeWithValue(ByteBuffer buf, int position, long time, Number value)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET, value);
}
void updateTimeWithNull(ByteBuffer buf, int position, long time)
@ -89,11 +100,28 @@ public abstract class NumericFirstBufferAggregator<TSelector extends BaseNullabl
if (timeSelector.isNull()) {
return;
}
long time = timeSelector.getLong();
long firstTime = buf.getLong(position);
if (needsFoldCheck) {
final Object object = valueSelector.getObject();
if (object instanceof SerializablePair) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) object;
if (inPair.lhs < firstTime) {
if (inPair.rhs == null) {
updateTimeWithNull(buf, position, inPair.lhs);
} else {
updateTimeWithValue(buf, position, inPair.lhs, inPair.rhs);
}
}
return;
}
}
long time = timeSelector.getLong();
if (time < firstTime) {
if (useDefault || !valueSelector.isNull()) {
updateTimeWithValue(buf, position, time);
updateTimeWithValue(buf, position, time, valueSelector);
} else {
updateTimeWithNull(buf, position, time);
}

View File

@ -19,8 +19,10 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -33,14 +35,16 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
{
static final int NULL_OFFSET = Long.BYTES;
static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
final VectorObjectSelector objectSelector;
final VectorValueSelector valueSelector;
private final boolean useDefault = NullHandling.replaceWithDefault();
private final VectorValueSelector timeSelector;
private long firstTime;
public NumericFirstVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
NumericFirstVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector, VectorObjectSelector objectSelector)
{
this.timeSelector = timeSelector;
this.objectSelector = objectSelector;
this.valueSelector = valueSelector;
firstTime = Long.MAX_VALUE;
}
@ -58,7 +62,16 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
{
final long[] timeVector = timeSelector.getLongVector();
final boolean[] nullTimeVector = timeSelector.getNullVector();
final boolean[] nullValueVector = valueSelector.getNullVector();
Object[] objectsWhichMightBeNumeric = null;
boolean[] nullValueVector = null;
if (objectSelector != null) {
objectsWhichMightBeNumeric = objectSelector.getObjectVector();
} else if (valueSelector != null) {
nullValueVector = valueSelector.getNullVector();
}
firstTime = buf.getLong(position);
// the time vector is already sorted
@ -68,22 +81,39 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
// A possible optimization here is to have 2 paths one for earliest where
// we can take advantage of the sorted nature of time
// and the earliest_by where we have to go over all elements.
int index;
int index;
for (int i = startRow; i < endRow; i++) {
if (nullTimeVector != null && nullTimeVector[i]) {
continue;
}
if (timeVector[i] >= firstTime) {
continue;
}
index = i;
if (nullTimeVector != null && nullTimeVector[index]) {
continue;
}
final long earliestTime = timeVector[index];
if (earliestTime >= firstTime) {
continue;
}
firstTime = earliestTime;
if (useDefault || nullValueVector == null || !nullValueVector[index]) {
updateTimeWithValue(buf, position, firstTime, index);
if (objectsWhichMightBeNumeric != null) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) objectsWhichMightBeNumeric[index];
if (inPair.lhs != null && inPair.lhs < firstTime) {
firstTime = inPair.lhs;
if (useDefault || inPair.rhs != null) {
updateTimeWithValue(buf, position, firstTime, index);
} else {
updateTimeWithNull(buf, position, firstTime);
}
}
} else {
updateTimeWithNull(buf, position, firstTime);
final long earliestTime = timeVector[index];
if (earliestTime < firstTime) {
firstTime = earliestTime;
if (useDefault || nullValueVector == null || !nullValueVector[index]) {
updateTimeWithValue(buf, position, earliestTime, index);
} else {
updateTimeWithNull(buf, position, earliestTime);
}
}
}
}
}
@ -110,18 +140,39 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
int positionOffset
)
{
boolean[] nulls = useDefault ? null : valueSelector.getNullVector();
long[] timeVector = timeSelector.getLongVector();
final long[] timeVector = timeSelector.getLongVector();
Object[] objectsWhichMightBeNumeric = null;
boolean[] nulls = null;
if (objectSelector != null) {
objectsWhichMightBeNumeric = objectSelector.getObjectVector();
} else if (valueSelector != null) {
nulls = useDefault ? null : valueSelector.getNullVector();
}
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) {
if (useDefault || nulls == null || !nulls[row]) {
updateTimeWithValue(buf, position, timeVector[row], row);
} else {
updateTimeWithNull(buf, position, timeVector[row]);
firstTime = buf.getLong(position);
if (objectsWhichMightBeNumeric != null) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) objectsWhichMightBeNumeric[row];
if (useDefault || inPair != null) {
if (inPair.lhs != null && inPair.lhs < firstTime) {
if (inPair.rhs != null) {
updateTimeWithValue(buf, position, inPair.lhs, row);
} else {
updateTimeWithNull(buf, position, inPair.lhs);
}
}
}
} else {
if (timeVector[row] < firstTime) {
if (useDefault || nulls == null || !nulls[row]) {
updateTimeWithValue(buf, position, timeVector[row], row);
} else {
updateTimeWithNull(buf, position, timeVector[row]);
}
}
}
}
@ -132,8 +183,8 @@ public abstract class NumericFirstVectorAggregator implements VectorAggregator
*
* @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 time the time to be updated in the buffer as the first time
* @param index he index of the vectorized vector which is the first value
*/
void updateTimeWithValue(ByteBuffer buf, int position, long time, int index)
{

View File

@ -165,7 +165,7 @@ public class StringFirstAggregatorFactory extends AggregatorFactory
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
FirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName), SerializablePairLongString.class)
);
}
}
@ -181,7 +181,7 @@ public class StringFirstAggregatorFactory extends AggregatorFactory
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
FirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName), SerializablePairLongString.class)
);
}
}
@ -245,7 +245,7 @@ public class StringFirstAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
return new StringFirstAggregateCombiner();
return new GenericFirstAggregateCombiner(SerializablePairLongString.class);
}
@Override

View File

@ -24,9 +24,6 @@ import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
@ -37,43 +34,6 @@ public class StringFirstLastUtils
{
private static final int NULL_VALUE = -1;
/**
* Returns whether a given value selector *might* contain SerializablePairLongString objects.
*/
public static boolean selectorNeedsFoldCheck(
final BaseObjectColumnValueSelector<?> valueSelector,
@Nullable final ColumnCapabilities valueSelectorCapabilities
)
{
if (valueSelectorCapabilities != null && !valueSelectorCapabilities.is(ValueType.COMPLEX)) {
// Known, non-complex type.
return false;
}
if (valueSelector instanceof NilColumnValueSelector) {
// Nil column, definitely no SerializablePairLongStrings.
return false;
}
// Check if the selector class could possibly be a SerializablePairLongString (either a superclass or subclass).
final Class<?> clazz = valueSelector.classOfObject();
return clazz.isAssignableFrom(SerializablePairLongString.class)
|| SerializablePairLongString.class.isAssignableFrom(clazz);
}
/**
* Returns whether an object *might* contain SerializablePairLongString objects.
*/
public static boolean objectNeedsFoldCheck(Object obj)
{
if (obj == null) {
return false;
}
final Class<?> clazz = obj.getClass();
return clazz.isAssignableFrom(SerializablePairLongString.class)
|| SerializablePairLongString.class.isAssignableFrom(clazz);
}
/**
* Return the object at a particular index from the vector selectors.
* index of bounds issues is the responsibility of the caller

View File

@ -76,7 +76,7 @@ public class StringFirstVectorAggregator implements VectorAggregator
continue;
}
index = i;
final boolean foldNeeded = StringFirstLastUtils.objectNeedsFoldCheck(objectsWhichMightBeStrings[index]);
final boolean foldNeeded = FirstLastUtils.objectNeedsFoldCheck(objectsWhichMightBeStrings[index], SerializablePairLongString.class);
if (foldNeeded) {
final SerializablePairLongString inPair = StringFirstLastUtils.readPairFromVectorSelectorsAtIndex(
timeSelector,
@ -125,7 +125,7 @@ public class StringFirstVectorAggregator implements VectorAggregator
if (obj == null) {
continue;
} else {
foldNeeded = StringFirstLastUtils.objectNeedsFoldCheck(obj);
foldNeeded = FirstLastUtils.objectNeedsFoldCheck(obj, SerializablePairLongString.class);
break;
}
}

View File

@ -19,30 +19,36 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class DoubleLastAggregator extends NumericLastAggregator<BaseDoubleColumnValueSelector>
public class DoubleLastAggregator extends NumericLastAggregator
{
double lastValue;
public DoubleLastAggregator(BaseLongColumnValueSelector timeSelector, BaseDoubleColumnValueSelector valueSelector)
public DoubleLastAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
lastValue = 0;
}
@Override
void setCurrentValue()
void setLastValue()
{
lastValue = valueSelector.getDouble();
}
@Override
void setLastValue(Number lastValue)
{
this.lastValue = lastValue.doubleValue();
}
@Override
public Object get()
{
return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
return new SerializablePairLongDouble(lastTime, rhsNull ? null : lastValue);
}
@Override

View File

@ -21,21 +21,20 @@ package org.apache.druid.query.aggregation.last;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.query.aggregation.SerializablePairLongDoubleComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.aggregation.first.DoubleFirstAggregatorFactory;
import org.apache.druid.query.aggregation.first.FirstLastUtils;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -45,6 +44,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -55,11 +55,14 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
@JsonTypeName("doubleLast")
public class DoubleLastAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongDoubleComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new DoubleLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -71,7 +74,8 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new DoubleLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -104,13 +108,18 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new DoubleLastAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongDouble.class
)
);
}
}
@ -126,26 +135,31 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
VectorColumnSelectorFactory columnSelectorFactory
)
{
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
return new DoubleLastVectorAggregator(timeSelector, valueSelector);
} else {
return NilVectorAggregator.of(new SerializablePair<>(0L, NullHandling.defaultDoubleValue()));
}
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new DoubleLastVectorAggregator(timeSelector, objectSelector);
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new DoubleLastBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongDouble.class
)
);
}
}
@ -178,74 +192,13 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("DoubleLastAggregatorFactory is not supported during ingestion for rollup");
return new GenericLastAggregateCombiner(SerializablePairLongDouble.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new DoubleLastAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
return new DoubleLastAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Double> pair = selector.getObject();
if (pair.lhs >= lastTime) {
lastTime = pair.lhs;
if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Double>> selector =
metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Double> pair = selector.getObject();
long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new DoubleLastAggregatorFactory(name, name, timeColumn);
}
@Override
@ -253,16 +206,16 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongDouble(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
return new SerializablePairLongDouble(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).doubleValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Double>) object).rhs;
return object == null ? null : ((SerializablePairLongDouble) object).rhs;
}
@Override
@ -302,8 +255,7 @@ public class DoubleLastAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE;
return TYPE;
}
@Override

View File

@ -19,20 +19,21 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class DoubleLastBufferAggregator extends NumericLastBufferAggregator<BaseDoubleColumnValueSelector>
public class DoubleLastBufferAggregator extends NumericLastBufferAggregator
{
public DoubleLastBufferAggregator(
BaseLongColumnValueSelector timeSelector,
BaseDoubleColumnValueSelector valueSelector
ColumnValueSelector valueSelector,
boolean needsFoldCheck
)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -42,16 +43,22 @@ public class DoubleLastBufferAggregator extends NumericLastBufferAggregator<Base
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSelector)
{
buf.putDouble(position, valueSelector.getDouble());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putDouble(position, value.doubleValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
return new SerializablePairLongDouble(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -32,16 +33,22 @@ public class DoubleLastVectorAggregator extends NumericLastVectorAggregator
{
double lastValue;
public DoubleLastVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, null, objectSelector);
lastValue = 0;
}
public DoubleLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, null);
lastValue = 0;
}
@Override
void putValue(ByteBuffer buf, int position, int index)
{
lastValue = valueSelector.getDoubleVector()[index];
lastValue = valueSelector != null ? valueSelector.getDoubleVector()[index] : ((SerializablePairLongDouble) objectSelector.getObjectVector()[index]).getRhs();
buf.putDouble(position, lastValue);
}
@ -57,7 +64,7 @@ public class DoubleLastVectorAggregator extends NumericLastVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
return new SerializablePairLongDouble(buf.getLong(position), rhsNull ? null : buf.getDouble(position + VALUE_OFFSET));
}
}

View File

@ -19,30 +19,37 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class FloatLastAggregator extends NumericLastAggregator<BaseFloatColumnValueSelector>
public class FloatLastAggregator extends NumericLastAggregator
{
float lastValue;
public FloatLastAggregator(BaseLongColumnValueSelector timeSelector, BaseFloatColumnValueSelector valueSelector)
public FloatLastAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
lastValue = 0;
}
@Override
void setCurrentValue()
void setLastValue()
{
lastValue = valueSelector.getFloat();
}
@Override
void setLastValue(Number lastValue)
{
this.lastValue = lastValue.floatValue();
}
@Override
public Object get()
{
return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
return new SerializablePairLongFloat(lastTime, rhsNull ? null : lastValue);
}
@Override

View File

@ -21,21 +21,20 @@ package org.apache.druid.query.aggregation.last;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.query.aggregation.SerializablePairLongFloatComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.aggregation.first.FirstLastUtils;
import org.apache.druid.query.aggregation.first.FloatFirstAggregatorFactory;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -45,6 +44,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -55,11 +55,14 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
@JsonTypeName("floatLast")
public class FloatLastAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongFloatComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new FloatLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -71,7 +74,8 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new FloatLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -102,13 +106,18 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new FloatLastAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongFloat.class
)
);
}
}
@ -116,13 +125,18 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseFloatColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new FloatLastBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongFloat.class
)
);
}
}
@ -138,14 +152,14 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
VectorColumnSelectorFactory columnSelectorFactory
)
{
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
return new FloatLastVectorAggregator(timeSelector, valueSelector);
} else {
return NilVectorAggregator.of(new SerializablePair<>(0L, NullHandling.defaultFloatValue()));
}
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new FloatLastVectorAggregator(timeSelector, objectSelector);
}
@Override
@ -176,72 +190,13 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("FloatLastAggregatorFactory is not supported during ingestion for rollup");
return new GenericLastAggregateCombiner(SerializablePairLongFloat.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new FloatLastAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
return new FloatLastAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Float> pair = selector.getObject();
if (pair.lhs >= lastTime) {
lastTime = pair.lhs;
if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
ColumnValueSelector<SerializablePair<Long, Float>> selector = metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Float> pair = selector.getObject();
long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new FloatLastAggregatorFactory(name, name, timeColumn);
}
@Override
@ -249,16 +204,16 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongFloat(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
return new SerializablePairLongFloat(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).floatValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Float>) object).rhs;
return object == null ? null : ((SerializablePairLongFloat) object).rhs;
}
@Override
@ -299,8 +254,7 @@ public class FloatLastAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return ColumnType.FLOAT;
return TYPE;
}
@Override

View File

@ -19,20 +19,22 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class FloatLastBufferAggregator extends NumericLastBufferAggregator<BaseFloatColumnValueSelector>
public class FloatLastBufferAggregator extends NumericLastBufferAggregator
{
public FloatLastBufferAggregator(
BaseLongColumnValueSelector timeSelector,
BaseFloatColumnValueSelector valueSelector
ColumnValueSelector valueSelector,
boolean needsFoldCheck
)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -42,16 +44,22 @@ public class FloatLastBufferAggregator extends NumericLastBufferAggregator<BaseF
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSelector)
{
buf.putFloat(position, valueSelector.getFloat());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putFloat(position, value.floatValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
return new SerializablePairLongFloat(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -32,20 +33,26 @@ public class FloatLastVectorAggregator extends NumericLastVectorAggregator
{
float lastValue;
public FloatLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
public FloatLastVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, null, objectSelector);
lastValue = 0;
}
public FloatLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector, null);
lastValue = 0;
}
@Override
void putValue(ByteBuffer buf, int position, int index)
{
lastValue = valueSelector.getFloatVector()[index];
lastValue = valueSelector != null ? valueSelector.getFloatVector()[index] : ((SerializablePairLongFloat) objectSelector.getObjectVector()[index]).getRhs();
buf.putFloat(position, lastValue);
}
@Override
public void initValue(ByteBuffer buf, int position)
{
@ -58,7 +65,7 @@ public class FloatLastVectorAggregator extends NumericLastVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
return new SerializablePairLongFloat(buf.getLong(position), rhsNull ? null : buf.getFloat(position + VALUE_OFFSET));
}
}

View File

@ -19,42 +19,48 @@
package org.apache.druid.query.aggregation.last;
import com.google.common.primitives.Longs;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.ObjectAggregateCombiner;
import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.query.aggregation.first.StringFirstAggregatorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
public class StringLastAggregateCombiner extends ObjectAggregateCombiner<SerializablePairLongString>
public class GenericLastAggregateCombiner<T extends SerializablePair<Long, ?>> extends ObjectAggregateCombiner<T>
{
private SerializablePairLongString lastValue;
private T lastValue;
private final Class<T> pairClass;
public GenericLastAggregateCombiner(Class<T> pairClass)
{
this.pairClass = pairClass;
}
@Override
public void reset(ColumnValueSelector selector)
{
lastValue = (SerializablePairLongString) selector.getObject();
lastValue = (T) selector.getObject();
}
@Override
public void fold(ColumnValueSelector selector)
{
SerializablePairLongString newValue = (SerializablePairLongString) selector.getObject();
if (StringFirstAggregatorFactory.TIME_COMPARATOR.compare(lastValue, newValue) < 0) {
lastValue = (SerializablePairLongString) selector.getObject();
T newValue = (T) selector.getObject();
if (Longs.compare(lastValue.lhs, newValue.lhs) < 0) {
lastValue = newValue;
}
}
@Nullable
@Override
public SerializablePairLongString getObject()
public T getObject()
{
return lastValue;
}
@Override
public Class<SerializablePairLongString> classOfObject()
public Class<? extends T> classOfObject()
{
return SerializablePairLongString.class;
return pairClass;
}
}

View File

@ -19,29 +19,36 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
public class LongLastAggregator extends NumericLastAggregator<BaseLongColumnValueSelector>
public class LongLastAggregator extends NumericLastAggregator
{
long lastValue;
public LongLastAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
public LongLastAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
lastValue = 0;
}
@Override
void setCurrentValue()
void setLastValue()
{
lastValue = valueSelector.getLong();
}
@Override
void setLastValue(Number lastValue)
{
this.lastValue = lastValue.longValue();
}
@Override
public Object get()
{
return new SerializablePair<>(lastTime, rhsNull ? null : lastValue);
return new SerializablePairLongLong(lastTime, rhsNull ? null : lastValue);
}
@Override

View File

@ -21,21 +21,20 @@ package org.apache.druid.query.aggregation.last;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.query.aggregation.SerializablePairLongLongComplexMetricSerde;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.any.NilVectorAggregator;
import org.apache.druid.query.aggregation.first.FirstLastUtils;
import org.apache.druid.query.aggregation.first.LongFirstAggregatorFactory;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -45,6 +44,7 @@ import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -55,11 +55,15 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
@JsonTypeName("longLast")
public class LongLastAggregatorFactory extends AggregatorFactory
{
public static final ColumnType TYPE = ColumnType.ofComplex(SerializablePairLongLongComplexMetricSerde.TYPE_NAME);
private static final Aggregator NIL_AGGREGATOR = new LongLastAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -71,7 +75,8 @@ public class LongLastAggregatorFactory extends AggregatorFactory
private static final BufferAggregator NIL_BUFFER_AGGREGATOR = new LongLastBufferAggregator(
NilColumnValueSelector.instance(),
NilColumnValueSelector.instance()
NilColumnValueSelector.instance(),
false
)
{
@Override
@ -102,13 +107,18 @@ public class LongLastAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_AGGREGATOR;
} else {
return new LongLastAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongLong.class
)
);
}
}
@ -116,13 +126,18 @@ public class LongLastAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final BaseLongColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
final ColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(fieldName);
if (valueSelector instanceof NilColumnValueSelector) {
return NIL_BUFFER_AGGREGATOR;
} else {
return new LongLastBufferAggregator(
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector
valueSelector,
FirstLastUtils.selectorNeedsFoldCheck(
valueSelector,
metricFactory.getColumnCapabilities(fieldName),
SerializablePairLongLong.class
)
);
}
}
@ -138,14 +153,14 @@ public class LongLastAggregatorFactory extends AggregatorFactory
VectorColumnSelectorFactory columnSelectorFactory
)
{
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName);
if (Types.isNumeric(capabilities)) {
VectorValueSelector valueSelector = columnSelectorFactory.makeValueSelector(fieldName);
VectorValueSelector timeSelector = columnSelectorFactory.makeValueSelector(timeColumn);
return new LongLastVectorAggregator(timeSelector, valueSelector);
} else {
return NilVectorAggregator.of(new SerializablePair<>(0L, NullHandling.defaultLongValue()));
}
VectorObjectSelector objectSelector = columnSelectorFactory.makeObjectSelector(fieldName);
return new LongLastVectorAggregator(timeSelector, objectSelector);
}
@Override
@ -176,72 +191,13 @@ public class LongLastAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
throw new UOE("LongLastAggregatorFactory is not supported during ingestion for rollup");
return new GenericLastAggregateCombiner(SerializablePairLongLong.class);
}
@Override
public AggregatorFactory getCombiningFactory()
{
return new LongLastAggregatorFactory(name, name, timeColumn)
{
@Override
public Aggregator factorize(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
return new LongLastAggregator(null, null)
{
@Override
public void aggregate()
{
SerializablePair<Long, Long> pair = selector.getObject();
if (pair.lhs >= lastTime) {
lastTime = pair.lhs;
if (pair.rhs != null) {
lastValue = pair.rhs;
rhsNull = false;
} else {
rhsNull = true;
}
}
}
};
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
{
final ColumnValueSelector<SerializablePair<Long, Long>> selector = metricFactory.makeColumnValueSelector(name);
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
public void aggregate(ByteBuffer buf, int position)
{
SerializablePair<Long, Long> pair = selector.getObject();
long lastTime = buf.getLong(position);
if (pair.lhs >= lastTime) {
if (pair.rhs != null) {
updateTimeWithValue(buf, position, pair.lhs);
} else {
updateTimeWithNull(buf, position, pair.lhs);
}
}
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
}
};
}
};
return new LongLastAggregatorFactory(name, name, timeColumn);
}
@Override
@ -249,16 +205,16 @@ public class LongLastAggregatorFactory extends AggregatorFactory
{
Map map = (Map) object;
if (map.get("rhs") == null) {
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), null);
return new SerializablePairLongLong(((Number) map.get("lhs")).longValue(), null);
}
return new SerializablePair<>(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
return new SerializablePairLongLong(((Number) map.get("lhs")).longValue(), ((Number) map.get("rhs")).longValue());
}
@Override
@Nullable
public Object finalizeComputation(@Nullable Object object)
{
return object == null ? null : ((SerializablePair<Long, Long>) object).rhs;
return object == null ? null : ((SerializablePairLongLong) object).rhs;
}
@Override
@ -298,8 +254,7 @@ public class LongLastAggregatorFactory extends AggregatorFactory
@Override
public ColumnType getIntermediateType()
{
// if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde
return ColumnType.LONG;
return TYPE;
}
@Override

View File

@ -19,16 +19,17 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
public class LongLastBufferAggregator extends NumericLastBufferAggregator<BaseLongColumnValueSelector>
public class LongLastBufferAggregator extends NumericLastBufferAggregator
{
public LongLastBufferAggregator(BaseLongColumnValueSelector timeSelector, BaseLongColumnValueSelector valueSelector)
public LongLastBufferAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, needsFoldCheck);
}
@Override
@ -38,16 +39,22 @@ public class LongLastBufferAggregator extends NumericLastBufferAggregator<BaseLo
}
@Override
void putValue(ByteBuffer buf, int position)
void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSelector)
{
buf.putLong(position, valueSelector.getLong());
}
@Override
void putValue(ByteBuffer buf, int position, Number value)
{
buf.putLong(position, value.longValue());
}
@Override
public Object get(ByteBuffer buf, int position)
{
boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
return new SerializablePairLongLong(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
}
@Override

View File

@ -19,7 +19,8 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.query.aggregation.SerializablePairLongLong;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -32,9 +33,15 @@ public class LongLastVectorAggregator extends NumericLastVectorAggregator
{
long lastValue;
public LongLastVectorAggregator(VectorValueSelector timeSelector, VectorObjectSelector objectSelector)
{
super(timeSelector, null, objectSelector);
lastValue = 0;
}
public LongLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
{
super(timeSelector, valueSelector);
super(timeSelector, valueSelector, null);
lastValue = 0;
}
@ -44,11 +51,10 @@ public class LongLastVectorAggregator extends NumericLastVectorAggregator
buf.putLong(position, 0);
}
@Override
void putValue(ByteBuffer buf, int position, int index)
{
lastValue = valueSelector.getLongVector()[index];
lastValue = valueSelector != null ? valueSelector.getLongVector()[index] : ((SerializablePairLongLong) objectSelector.getObjectVector()[index]).getRhs();
buf.putLong(position, lastValue);
}
@ -61,6 +67,6 @@ public class LongLastVectorAggregator extends NumericLastVectorAggregator
public Object get(ByteBuffer buf, int position)
{
final boolean rhsNull = isValueNull(buf, position);
return new SerializablePair<>(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
return new SerializablePairLongLong(buf.getLong(position), rhsNull ? null : buf.getLong(position + VALUE_OFFSET));
}
}

View File

@ -19,47 +19,78 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
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;
import org.apache.druid.segment.ColumnValueSelector;
/**
* 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
public abstract class NumericLastAggregator implements Aggregator
{
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
private final BaseLongColumnValueSelector timeSelector;
final ColumnValueSelector valueSelector;
final boolean needsFoldCheck;
long lastTime;
boolean rhsNull;
public NumericLastAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
public NumericLastAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
this.needsFoldCheck = needsFoldCheck;
lastTime = Long.MIN_VALUE;
rhsNull = !useDefault;
}
/**
* Store the current primitive typed 'last' value
*/
abstract void setLastValue();
abstract void setLastValue(Number lastValue);
@Override
public void aggregate()
{
if (timeSelector.isNull()) {
return;
}
if (needsFoldCheck) {
final Object object = valueSelector.getObject();
if (object instanceof SerializablePair) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) object;
if (inPair.lhs >= lastTime) {
lastTime = inPair.lhs;
if (inPair.rhs == null) {
rhsNull = true;
} else {
rhsNull = false;
setLastValue(inPair.rhs);
}
}
return;
}
}
long time = timeSelector.getLong();
if (time >= lastTime) {
lastTime = time;
if (useDefault || !valueSelector.isNull()) {
setCurrentValue();
setLastValue();
rhsNull = false;
} else {
setLastValue(0);
rhsNull = true;
}
}
@ -70,9 +101,4 @@ public abstract class NumericLastAggregator<TSelector extends BaseNullableColumn
{
// nothing to close
}
/**
* Store the current primitive typed 'last' value
*/
abstract void setCurrentValue();
}

View File

@ -19,11 +19,12 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
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 org.apache.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
@ -33,8 +34,7 @@ import java.nio.ByteBuffer;
* 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
public abstract class NumericLastBufferAggregator implements BufferAggregator
{
static final int NULL_OFFSET = Long.BYTES;
static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
@ -42,12 +42,14 @@ public abstract class NumericLastBufferAggregator<TSelector extends BaseNullable
private final boolean useDefault = NullHandling.replaceWithDefault();
private final BaseLongColumnValueSelector timeSelector;
final TSelector valueSelector;
final ColumnValueSelector valueSelector;
final boolean needsFoldCheck;
public NumericLastBufferAggregator(BaseLongColumnValueSelector timeSelector, TSelector valueSelector)
public NumericLastBufferAggregator(BaseLongColumnValueSelector timeSelector, ColumnValueSelector valueSelector, boolean needsFoldCheck)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
this.needsFoldCheck = needsFoldCheck;
}
/**
@ -58,18 +60,27 @@ public abstract class NumericLastBufferAggregator<TSelector extends BaseNullable
/**
* Place the primitive value in the buffer at the position of {@link #VALUE_OFFSET}
*/
abstract void putValue(ByteBuffer buf, int position);
abstract void putValue(ByteBuffer buf, int position, ColumnValueSelector valueSelector);
abstract void putValue(ByteBuffer buf, int position, Number value);
boolean isValueNull(ByteBuffer buf, int position)
{
return buf.get(position + NULL_OFFSET) == NullHandling.IS_NULL_BYTE;
}
void updateTimeWithValue(ByteBuffer buf, int position, long time)
void updateTimeWithValue(ByteBuffer buf, int position, long time, ColumnValueSelector valueSelector)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET);
putValue(buf, position + VALUE_OFFSET, valueSelector);
}
void updateTimeWithValue(ByteBuffer buf, int position, long time, Number value)
{
buf.putLong(position, time);
buf.put(position + NULL_OFFSET, NullHandling.IS_NOT_NULL_BYTE);
putValue(buf, position + VALUE_OFFSET, value);
}
void updateTimeWithNull(ByteBuffer buf, int position, long time)
@ -92,11 +103,29 @@ public abstract class NumericLastBufferAggregator<TSelector extends BaseNullable
if (timeSelector.isNull()) {
return;
}
long time = timeSelector.getLong();
long lastTime = buf.getLong(position);
if (needsFoldCheck) {
final Object object = valueSelector.getObject();
if (object instanceof SerializablePair) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) object;
if (inPair.lhs >= lastTime) {
if (inPair.rhs == null) {
updateTimeWithNull(buf, position, inPair.lhs);
} else {
updateTimeWithValue(buf, position, inPair.lhs, inPair.rhs);
}
}
return;
}
}
long time = timeSelector.getLong();
if (time >= lastTime) {
if (useDefault || !valueSelector.isNull()) {
updateTimeWithValue(buf, position, time);
updateTimeWithValue(buf, position, time, valueSelector);
} else {
updateTimeWithNull(buf, position, time);
}

View File

@ -19,8 +19,10 @@
package org.apache.druid.query.aggregation.last;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.segment.vector.VectorObjectSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
@ -34,14 +36,17 @@ public abstract class NumericLastVectorAggregator implements VectorAggregator
static final int NULL_OFFSET = Long.BYTES;
static final int VALUE_OFFSET = NULL_OFFSET + Byte.BYTES;
final VectorValueSelector valueSelector;
final VectorObjectSelector objectSelector;
private final boolean useDefault = NullHandling.replaceWithDefault();
private final VectorValueSelector timeSelector;
private long lastTime;
public NumericLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector)
NumericLastVectorAggregator(VectorValueSelector timeSelector, VectorValueSelector valueSelector, VectorObjectSelector objectSelector)
{
this.timeSelector = timeSelector;
this.valueSelector = valueSelector;
this.objectSelector = objectSelector;
lastTime = Long.MIN_VALUE;
}
@ -56,13 +61,23 @@ public abstract class NumericLastVectorAggregator implements VectorAggregator
@Override
public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
{
if (timeSelector == null) {
return;
}
final long[] timeVector = timeSelector.getLongVector();
final boolean[] nullValueVector = valueSelector.getNullVector();
Object[] objectsWhichMightBeNumeric = null;
boolean[] nullValueVector = null;
boolean nullAbsent = false;
if (objectSelector != null) {
objectsWhichMightBeNumeric = objectSelector.getObjectVector();
} else if (valueSelector != null) {
nullValueVector = valueSelector.getNullVector();
}
lastTime = 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
if (nullValueVector == null) {
nullAbsent = true;
}
@ -79,14 +94,25 @@ public abstract class NumericLastVectorAggregator implements VectorAggregator
}
}
//find the first non-null value
final long latestTime = timeVector[index];
if (latestTime >= lastTime) {
lastTime = latestTime;
if (useDefault || nullValueVector == null || !nullValueVector[index]) {
updateTimeWithValue(buf, position, lastTime, index);
} else {
updateTimeWithNull(buf, position, lastTime);
if (objectsWhichMightBeNumeric != null) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) objectsWhichMightBeNumeric[index];
if (inPair.lhs != null && inPair.lhs >= lastTime) {
lastTime = inPair.lhs;
if (useDefault || inPair.rhs != null) {
updateTimeWithValue(buf, position, lastTime, index);
} else {
updateTimeWithNull(buf, position, lastTime);
}
}
} else {
final long latestTime = timeVector[index];
if (latestTime >= lastTime) {
lastTime = latestTime;
if (useDefault || nullValueVector == null || !nullValueVector[index]) {
updateTimeWithValue(buf, position, lastTime, index);
} else {
updateTimeWithNull(buf, position, lastTime);
}
}
}
}
@ -113,19 +139,45 @@ public abstract class NumericLastVectorAggregator implements VectorAggregator
int positionOffset
)
{
if (timeSelector == null) {
return;
}
final long[] timeVector = timeSelector.getLongVector();
Object[] objectsWhichMightBeNumeric = null;
boolean[] nulls = null;
if (objectSelector != null) {
objectsWhichMightBeNumeric = objectSelector.getObjectVector();
} else if (valueSelector != null) {
nulls = useDefault ? null : valueSelector.getNullVector();
}
boolean[] nulls = useDefault ? null : valueSelector.getNullVector();
long[] timeVector = timeSelector.getLongVector();
for (int i = 0; i < numRows; i++) {
int position = positions[i] + positionOffset;
int row = rows == null ? i : rows[i];
long lastTime = buf.getLong(position);
if (timeVector[row] >= lastTime) {
if (useDefault || nulls == null || !nulls[row]) {
updateTimeWithValue(buf, position, timeVector[row], row);
} else {
updateTimeWithNull(buf, position, timeVector[row]);
if (objectsWhichMightBeNumeric != null) {
final SerializablePair<Long, Number> inPair = (SerializablePair<Long, Number>) objectsWhichMightBeNumeric[row];
if (useDefault || inPair != null) {
if (inPair.lhs != null && inPair.lhs >= lastTime) {
if (inPair.rhs != null) {
updateTimeWithValue(buf, position, inPair.lhs, row);
} else {
updateTimeWithNull(buf, position, inPair.lhs);
}
}
}
} else {
if (timeVector[row] >= lastTime) {
if (useDefault || nulls == null || !nulls[row]) {
updateTimeWithValue(buf, position, timeVector[row], row);
} else {
updateTimeWithNull(buf, position, timeVector[row]);
}
}
}
}
@ -136,7 +188,7 @@ public abstract class NumericLastVectorAggregator implements VectorAggregator
* @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 index he index of the vectorized vector which is the last value
*/
void updateTimeWithValue(ByteBuffer buf, int position, long time, int index)
{

View File

@ -32,8 +32,8 @@ import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.first.FirstLastUtils;
import org.apache.druid.query.aggregation.first.StringFirstAggregatorFactory;
import org.apache.druid.query.aggregation.first.StringFirstLastUtils;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
@ -130,7 +130,7 @@ public class StringLastAggregatorFactory extends AggregatorFactory
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
FirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName), SerializablePairLongString.class)
);
}
}
@ -146,7 +146,7 @@ public class StringLastAggregatorFactory extends AggregatorFactory
metricFactory.makeColumnValueSelector(timeColumn),
valueSelector,
maxStringBytes,
StringFirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName))
FirstLastUtils.selectorNeedsFoldCheck(valueSelector, metricFactory.getColumnCapabilities(fieldName), SerializablePairLongString.class)
);
}
}
@ -210,7 +210,7 @@ public class StringLastAggregatorFactory extends AggregatorFactory
@Override
public AggregateCombiner makeAggregateCombiner()
{
return new StringLastAggregateCombiner();
return new GenericLastAggregateCombiner(SerializablePairLongString.class);
}
@Override

View File

@ -22,6 +22,7 @@ package org.apache.druid.query.aggregation.last;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.query.aggregation.SerializablePairLongString;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.first.FirstLastUtils;
import org.apache.druid.query.aggregation.first.StringFirstLastUtils;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.vector.VectorObjectSelector;
@ -81,7 +82,7 @@ public class StringLastVectorAggregator implements VectorAggregator
continue;
}
index = i;
final boolean foldNeeded = StringFirstLastUtils.objectNeedsFoldCheck(objectsWhichMightBeStrings[index]);
final boolean foldNeeded = FirstLastUtils.objectNeedsFoldCheck(objectsWhichMightBeStrings[index], SerializablePairLongString.class);
if (foldNeeded) {
// Less efficient code path when folding is a possibility (we must read the value selector first just in case
// it's a foldable object).
@ -140,7 +141,7 @@ public class StringLastVectorAggregator implements VectorAggregator
boolean foldNeeded = false;
for (Object obj : objectsWhichMightBeStrings) {
if (obj != null) {
foldNeeded = StringFirstLastUtils.objectNeedsFoldCheck(obj);
foldNeeded = FirstLastUtils.objectNeedsFoldCheck(obj, SerializablePairLongString.class);
break;
}
}

View File

@ -210,21 +210,21 @@ public class AggregatorFactoryTest extends InitializedNullHandlingTest
.add("longSum", ColumnType.LONG)
.add("longMin", ColumnType.LONG)
.add("longMax", ColumnType.LONG)
.add("longFirst", ColumnType.LONG)
.add("longLast", ColumnType.LONG)
.add("longFirst", null)
.add("longLast", null)
.add("longAny", ColumnType.LONG)
.add("doubleSum", ColumnType.DOUBLE)
.add("doubleMin", ColumnType.DOUBLE)
.add("doubleMax", ColumnType.DOUBLE)
.add("doubleFirst", ColumnType.DOUBLE)
.add("doubleLast", ColumnType.DOUBLE)
.add("doubleFirst", null)
.add("doubleLast", null)
.add("doubleAny", ColumnType.DOUBLE)
.add("doubleMean", null)
.add("floatSum", ColumnType.FLOAT)
.add("floatMin", ColumnType.FLOAT)
.add("floatMax", ColumnType.FLOAT)
.add("floatFirst", ColumnType.FLOAT)
.add("floatLast", ColumnType.FLOAT)
.add("floatFirst", null)
.add("floatLast", null)
.add("floatAny", ColumnType.FLOAT)
.add("stringFirst", null)
.add("stringLast", null)

View File

@ -0,0 +1,350 @@
/*
* 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;
import com.google.common.primitives.Ints;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Random;
public class SerializablePairLongDoubleBufferStoreTest
{
private final Random random = new Random(0);
private static final int MIN_INTEGER = 100;
private static final long MIN_LONG = 0L;
private final SerializablePairLongDouble[] integerRangeArr = new SerializablePairLongDouble[]{
new SerializablePairLongDouble((long) MIN_INTEGER, 10D),
new SerializablePairLongDouble(101L, 20D),
new SerializablePairLongDouble(102L, 30D),
};
private final SerializablePairLongDouble[] longRangeArr = new SerializablePairLongDouble[]{
new SerializablePairLongDouble((long) MIN_LONG, 10D),
new SerializablePairLongDouble(101L, 20D),
new SerializablePairLongDouble(102L, 30D),
new SerializablePairLongDouble((long) Integer.MAX_VALUE, 40D),
new SerializablePairLongDouble(Long.MAX_VALUE, 50D),
};
private final SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
private SerializablePairLongDoubleBufferStore bufferStore;
@Before
public void setup() throws Exception
{
bufferStore = new SerializablePairLongDoubleBufferStore(
new SerializedStorage<>(
writeOutMedium.makeWriteOutBytes(),
new SerializablePairLongDoubleSimpleStagedSerde()
)
);
}
@Test
public void testIteratorSimple() throws Exception
{
for (SerializablePairLongDouble value : integerRangeArr) {
bufferStore.store(value);
}
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(integerRangeArr[i], iterator.next());
i++;
}
}
@Test
public void testIteratorNull() throws Exception
{
bufferStore.store(null);
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertNull(iterator.next());
}
@Test
public void testIteratorIdempotentHasNext() throws Exception
{
bufferStore.store(integerRangeArr[0]);
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
// expect hasNext() to not modify state
Assert.assertTrue(iterator.hasNext());
}
@Test(expected = NoSuchElementException.class)
public void testIteratorEmptyThrows() throws Exception
{
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
iterator.next();
}
@Test
public void testIteratorEmptyHasNext() throws Exception
{
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testMinValueUsesInteger() throws Exception
{
for (SerializablePairLongDouble value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(integerRangeArr[0].lhs.longValue(), columnHeader.getMinValue());
Assert.assertTrue(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesLong() throws Exception
{
for (SerializablePairLongDouble value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(MIN_LONG, columnHeader.getMinValue());
Assert.assertFalse(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesIntegerSerialization() throws Exception
{
for (SerializablePairLongDouble value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongDoubleColumnHeader deserializedColumnhHeader =
(SerializablePairLongDoubleColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongDouble.class);
Assert.assertEquals(MIN_INTEGER, deserializedColumnhHeader.getMinValue());
Assert.assertTrue(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testMinValueSerialization() throws Exception
{
for (SerializablePairLongDouble value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongDoubleColumnHeader deserializedColumnhHeader =
(SerializablePairLongDoubleColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongDouble.class);
Assert.assertEquals(MIN_LONG, deserializedColumnhHeader.getMinValue());
Assert.assertFalse(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testLargeBuffer() throws Exception
{
SerializablePairLongDouble value =
new SerializablePairLongDouble(Long.MAX_VALUE, Double.POSITIVE_INFINITY);
bufferStore.store(value);
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertEquals(value, iterator.next());
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testLargeValueCount() throws Exception
{
List<SerializablePairLongDouble> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongDouble(Integer.MAX_VALUE + (long) i, random.nextDouble()));
}
assertBufferedValuesEqual(valueList);
}
@Test
public void testOverflowTransfer() throws Exception
{
bufferStore.store(new SerializablePairLongDouble(Long.MIN_VALUE, 10D));
bufferStore.store(new SerializablePairLongDouble(Long.MAX_VALUE, 10D));
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(94, transferredBuffer.getSerializedSize());
}
@Test
public void testNullOnlyTransfer() throws Exception
{
bufferStore.store(null);
bufferStore.store(null);
bufferStore.store(null);
SerializablePairLongDoubleColumnHeader columnHeader = (SerializablePairLongDoubleColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(59, transferredBuffer.getSerializedSize());
}
@Test
public void testTransferIntegerRange() throws Exception
{
for (SerializablePairLongDouble value : integerRangeArr) {
bufferStore.store(value);
}
Assert.assertTrue(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(integerRangeArr);
}
@Test
public void testTransferLongRange() throws Exception
{
for (SerializablePairLongDouble value : longRangeArr) {
bufferStore.store(value);
}
Assert.assertFalse(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(longRangeArr);
}
private void assertBufferedValuesEqual(List<SerializablePairLongDouble> input) throws IOException
{
for (SerializablePairLongDouble pairLongLong : input) {
bufferStore.store(pairLongLong);
}
IOIterator<SerializablePairLongDouble> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(input.get(i), iterator.next());
i++;
}
Assert.assertEquals(
StringUtils.format("element count mismatch: expected %s, got %s", input.size(), i),
input.size(),
i
);
}
private void assertTransferredValuesEqual(SerializablePairLongDouble[] input) throws IOException
{
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer =
bufferStore.transferToRowWriter(NativeClearedByteBufferProvider.INSTANCE, writeOutMedium);
HeapByteBufferWriteOutBytes resultChannel = new HeapByteBufferWriteOutBytes();
transferredBuffer.writeTo(resultChannel, null);
try (SerializablePairLongDoubleComplexColumn column = createComplexColumn(transferredBuffer, resultChannel)) {
for (int i = 0; i < input.length; i++) {
Assert.assertEquals(input[i], column.getRowValue(i));
}
}
}
private static SerializablePairLongDoubleComplexColumn createComplexColumn(
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer,
HeapByteBufferWriteOutBytes resultChannel
)
{
ByteBuffer byteBuffer = ByteBuffer.allocate(Ints.checkedCast(transferredBuffer.getSerializedSize()));
resultChannel.readFully(0, byteBuffer);
byteBuffer.flip();
SerializablePairLongDoubleComplexMetricSerde complexMetricSerde = new SerializablePairLongDoubleComplexMetricSerde();
ColumnBuilder builder = new ColumnBuilder();
complexMetricSerde.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
SerializablePairLongDoubleComplexColumn column = (SerializablePairLongDoubleComplexColumn) columnHolder.getColumn();
return column;
}
}

View File

@ -0,0 +1,238 @@
/*
* 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;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicReference;
public class SerializablePairLongDoubleComplexMetricSerdeTest
{
static {
NullHandling.initializeForTests();
}
private static final SerializablePairLongDoubleComplexMetricSerde COMPRESSED_SERDE =
new SerializablePairLongDoubleComplexMetricSerde();
// want deterministic test input
private final Random random = new Random(0);
@Test
public void testSingle() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongDouble(100L, 10D)), 75);
}
@Test
public void testLargeRHS() throws Exception
{
// single entry spans more than one block in underlying storage
assertExpected(ImmutableList.of(new SerializablePairLongDouble(
100L,
random.nextDouble()
)), 78);
}
@Test
public void testCompressable() throws Exception
{
int numLongs = 10;
List<SerializablePairLongDouble> valueList = new ArrayList<>();
List<Double> doubleList = new ArrayList<>();
for (int i = 0; i < numLongs; i++) {
doubleList.add(random.nextDouble());
}
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongDouble(Integer.MAX_VALUE + (long) i, doubleList.get(i % numLongs)));
}
assertExpected(valueList, 80509);
}
@Test
public void testHighlyCompressable() throws Exception
{
List<SerializablePairLongDouble> valueList = new ArrayList<>();
Double doubleValue = random.nextDouble();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongDouble(Integer.MAX_VALUE + (long) i, doubleValue));
}
assertExpected(valueList, 80274);
}
@Test
public void testRandom() throws Exception
{
List<SerializablePairLongDouble> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongDouble(random.nextLong(), random.nextDouble()));
}
assertExpected(valueList, 210958);
}
@Test
public void testNullRHS() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongDouble(100L, null)), 71);
}
@Test
public void testEmpty() throws Exception
{
// minimum size for empty data
assertExpected(Collections.emptyList(), 57);
}
@Test
public void testSingleNull() throws Exception
{
assertExpected(Arrays.asList(new SerializablePairLongDouble[]{null}), 58);
}
@Test
public void testMultipleNull() throws Exception
{
assertExpected(Arrays.asList(null, null, null, null), 59);
}
private ByteBuffer assertExpected(
List<SerializablePairLongDouble> expected,
int expectedCompressedSize
) throws IOException
{
SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer(
expected,
COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used"),
expectedCompressedSize
).asReadOnlyBuffer();
try (ComplexColumn compressedCol = createComplexColumn(compressedBuffer)
) {
for (int i = 0; i < expected.size(); i++) {
Assert.assertEquals(expected.get(i), compressedCol.getRowValue(i));
}
}
return compressedBuffer;
}
private ComplexColumn createComplexColumn(ByteBuffer byteBuffer)
{
ColumnBuilder builder = new ColumnBuilder();
int serializedSize = byteBuffer.remaining();
COMPRESSED_SERDE.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
final ComplexColumn col = (ComplexColumn) columnHolder.getColumn();
if (col instanceof SerializablePairLongDoubleComplexColumn) {
Assert.assertEquals(serializedSize, col.getLength());
}
Assert.assertEquals("serializablePairLongDouble", col.getTypeName());
Assert.assertEquals(SerializablePairLongDouble.class, col.getClazz());
return col;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private static ByteBuffer serializeAllValuesToByteBuffer(
List<SerializablePairLongDouble> values,
GenericColumnSerializer serializer,
int expectedSize
) throws IOException
{
serializer.open();
final AtomicReference<SerializablePairLongDouble> reference = new AtomicReference<>(null);
ColumnValueSelector<SerializablePairLongDouble> valueSelector =
new SingleObjectColumnValueSelector<SerializablePairLongDouble>(
SerializablePairLongDouble.class
)
{
@Nullable
@Override
public SerializablePairLongDouble getObject()
{
return reference.get();
}
};
for (SerializablePairLongDouble selector : values) {
reference.set(selector);
serializer.serialize(valueSelector);
}
return serializeToByteBuffer(serializer, expectedSize);
}
private static ByteBuffer serializeToByteBuffer(
GenericColumnSerializer<SerializablePairLongDouble> serializer,
int expectedSize
) throws IOException
{
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
serializer.writeTo(channel, null);
ByteBuffer byteBuffer = ByteBuffer.allocate((int) channel.size()).order(ByteOrder.nativeOrder());
channel.readFully(0, byteBuffer);
byteBuffer.flip();
if (expectedSize > -1) {
Assert.assertEquals(expectedSize, serializer.getSerializedSize());
}
Assert.assertEquals(serializer.getSerializedSize(), byteBuffer.limit());
return byteBuffer;
}
}

View File

@ -0,0 +1,99 @@
/*
* 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;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Random;
public class SerializablePairLongDoubleDeltaEncodedStagedSerdeTest
{
private static final SerializablePairLongDoubleDeltaEncodedStagedSerde INTEGER_SERDE =
new SerializablePairLongDoubleDeltaEncodedStagedSerde(0L, true);
private static final SerializablePairLongDoubleDeltaEncodedStagedSerde LONG_SERDE =
new SerializablePairLongDoubleDeltaEncodedStagedSerde(0L, false);
private final Random random = new Random(0);
@Test
public void testNull()
{
assertValueEquals(null, 0, INTEGER_SERDE);
}
@Test
public void testSimpleInteger()
{
assertValueEquals(new SerializablePairLongDouble(100L, 1000000000000.12312312312D), 13, INTEGER_SERDE);
}
@Test
public void testNullRHSInteger()
{
assertValueEquals(new SerializablePairLongDouble(100L, null), 5, INTEGER_SERDE);
}
@Test
public void testLargeRHSInteger()
{
assertValueEquals(
new SerializablePairLongDouble(100L, random.nextDouble()),
13,
INTEGER_SERDE
);
}
@Test
public void testSimpleLong()
{
assertValueEquals(new SerializablePairLongDouble(100L, 1000000000000.12312312312D), 17, LONG_SERDE);
}
@Test
public void testNullRHSLong()
{
assertValueEquals(new SerializablePairLongDouble(100L, null), 9, LONG_SERDE);
}
@Test
public void testLargeRHSLong()
{
assertValueEquals(
new SerializablePairLongDouble(100L, random.nextDouble()),
17,
LONG_SERDE
);
}
private static void assertValueEquals(
@Nullable SerializablePairLongDouble value,
int size,
SerializablePairLongDoubleDeltaEncodedStagedSerde serde
)
{
byte[] bytes = serde.serialize(value);
Assert.assertEquals(size, bytes.length);
SerializablePairLongDouble deserialized = serde.deserialize(bytes);
Assert.assertEquals(value, deserialized);
}
}

View File

@ -0,0 +1,350 @@
/*
* 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;
import com.google.common.primitives.Ints;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Random;
public class SerializablePairLongFloatBufferStoreTest
{
private final Random random = new Random(0);
private static final int MIN_INTEGER = 100;
private static final long MIN_LONG = 0L;
private final SerializablePairLongFloat[] integerRangeArr = new SerializablePairLongFloat[]{
new SerializablePairLongFloat((long) MIN_INTEGER, 10F),
new SerializablePairLongFloat(101L, 20F),
new SerializablePairLongFloat(102L, 30F),
};
private final SerializablePairLongFloat[] longRangeArr = new SerializablePairLongFloat[]{
new SerializablePairLongFloat((long) MIN_LONG, 10F),
new SerializablePairLongFloat(101L, 20F),
new SerializablePairLongFloat(102L, 30F),
new SerializablePairLongFloat((long) Integer.MAX_VALUE, 40F),
new SerializablePairLongFloat(Long.MAX_VALUE, 50F),
};
private final SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
private SerializablePairLongFloatBufferStore bufferStore;
@Before
public void setup() throws Exception
{
bufferStore = new SerializablePairLongFloatBufferStore(
new SerializedStorage<>(
writeOutMedium.makeWriteOutBytes(),
new SerializablePairLongFloatSimpleStagedSerde()
)
);
}
@Test
public void testIteratorSimple() throws Exception
{
for (SerializablePairLongFloat value : integerRangeArr) {
bufferStore.store(value);
}
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(integerRangeArr[i], iterator.next());
i++;
}
}
@Test
public void testIteratorNull() throws Exception
{
bufferStore.store(null);
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertNull(iterator.next());
}
@Test
public void testIteratorIdempotentHasNext() throws Exception
{
bufferStore.store(integerRangeArr[0]);
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
// expect hasNext() to not modify state
Assert.assertTrue(iterator.hasNext());
}
@Test(expected = NoSuchElementException.class)
public void testIteratorEmptyThrows() throws Exception
{
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
iterator.next();
}
@Test
public void testIteratorEmptyHasNext() throws Exception
{
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testMinValueUsesInteger() throws Exception
{
for (SerializablePairLongFloat value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(integerRangeArr[0].lhs.longValue(), columnHeader.getMinValue());
Assert.assertTrue(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesLong() throws Exception
{
for (SerializablePairLongFloat value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(MIN_LONG, columnHeader.getMinValue());
Assert.assertFalse(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesIntegerSerialization() throws Exception
{
for (SerializablePairLongFloat value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongFloatColumnHeader deserializedColumnhHeader =
(SerializablePairLongFloatColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongFloat.class);
Assert.assertEquals(MIN_INTEGER, deserializedColumnhHeader.getMinValue());
Assert.assertTrue(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testMinValueSerialization() throws Exception
{
for (SerializablePairLongFloat value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongFloatColumnHeader deserializedColumnhHeader =
(SerializablePairLongFloatColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongFloat.class);
Assert.assertEquals(MIN_LONG, deserializedColumnhHeader.getMinValue());
Assert.assertFalse(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testLargeBuffer() throws Exception
{
SerializablePairLongFloat value =
new SerializablePairLongFloat(Long.MAX_VALUE, Float.POSITIVE_INFINITY);
bufferStore.store(value);
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertEquals(value, iterator.next());
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testLargeValueCount() throws Exception
{
List<SerializablePairLongFloat> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongFloat(Integer.MAX_VALUE + (long) i, random.nextFloat()));
}
assertBufferedValuesEqual(valueList);
}
@Test
public void testOverflowTransfer() throws Exception
{
bufferStore.store(new SerializablePairLongFloat(Long.MIN_VALUE, 10F));
bufferStore.store(new SerializablePairLongFloat(Long.MAX_VALUE, 10F));
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(90, transferredBuffer.getSerializedSize());
}
@Test
public void testNullOnlyTransfer() throws Exception
{
bufferStore.store(null);
bufferStore.store(null);
bufferStore.store(null);
SerializablePairLongFloatColumnHeader columnHeader = (SerializablePairLongFloatColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(59, transferredBuffer.getSerializedSize());
}
@Test
public void testTransferIntegerRange() throws Exception
{
for (SerializablePairLongFloat value : integerRangeArr) {
bufferStore.store(value);
}
Assert.assertTrue(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(integerRangeArr);
}
@Test
public void testTransferLongRange() throws Exception
{
for (SerializablePairLongFloat value : longRangeArr) {
bufferStore.store(value);
}
Assert.assertFalse(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(longRangeArr);
}
private void assertBufferedValuesEqual(List<SerializablePairLongFloat> input) throws IOException
{
for (SerializablePairLongFloat pairLongLong : input) {
bufferStore.store(pairLongLong);
}
IOIterator<SerializablePairLongFloat> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(input.get(i), iterator.next());
i++;
}
Assert.assertEquals(
StringUtils.format("element count mismatch: expected %s, got %s", input.size(), i),
input.size(),
i
);
}
private void assertTransferredValuesEqual(SerializablePairLongFloat[] input) throws IOException
{
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer =
bufferStore.transferToRowWriter(NativeClearedByteBufferProvider.INSTANCE, writeOutMedium);
HeapByteBufferWriteOutBytes resultChannel = new HeapByteBufferWriteOutBytes();
transferredBuffer.writeTo(resultChannel, null);
try (SerializablePairLongFloatComplexColumn column = createComplexColumn(transferredBuffer, resultChannel)) {
for (int i = 0; i < input.length; i++) {
Assert.assertEquals(input[i], column.getRowValue(i));
}
}
}
private static SerializablePairLongFloatComplexColumn createComplexColumn(
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer,
HeapByteBufferWriteOutBytes resultChannel
)
{
ByteBuffer byteBuffer = ByteBuffer.allocate(Ints.checkedCast(transferredBuffer.getSerializedSize()));
resultChannel.readFully(0, byteBuffer);
byteBuffer.flip();
SerializablePairLongFloatComplexMetricSerde complexMetricSerde = new SerializablePairLongFloatComplexMetricSerde();
ColumnBuilder builder = new ColumnBuilder();
complexMetricSerde.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
SerializablePairLongFloatComplexColumn column = (SerializablePairLongFloatComplexColumn) columnHolder.getColumn();
return column;
}
}

View File

@ -0,0 +1,238 @@
/*
* 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;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicReference;
public class SerializablePairLongFloatComplexMetricSerdeTest
{
static {
NullHandling.initializeForTests();
}
private static final SerializablePairLongFloatComplexMetricSerde COMPRESSED_SERDE =
new SerializablePairLongFloatComplexMetricSerde();
// want deterministic test input
private final Random random = new Random(0);
@Test
public void testSingle() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongFloat(100L, 10F)), 75);
}
@Test
public void testLargeRHS() throws Exception
{
// single entry spans more than one block in underlying storage
assertExpected(ImmutableList.of(new SerializablePairLongFloat(
100L,
random.nextFloat()
)), 75);
}
@Test
public void testCompressable() throws Exception
{
int numLongs = 10;
List<SerializablePairLongFloat> valueList = new ArrayList<>();
List<Float> floatList = new ArrayList<>();
for (int i = 0; i < numLongs; i++) {
floatList.add(random.nextFloat());
}
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongFloat(Integer.MAX_VALUE + (long) i, floatList.get(i % numLongs)));
}
assertExpected(valueList, 80418);
}
@Test
public void testHighlyCompressable() throws Exception
{
List<SerializablePairLongFloat> valueList = new ArrayList<>();
Float floatValue = random.nextFloat();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongFloat(Integer.MAX_VALUE + (long) i, floatValue));
}
assertExpected(valueList, 80260);
}
@Test
public void testRandom() throws Exception
{
List<SerializablePairLongFloat> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongFloat(random.nextLong(), random.nextFloat()));
}
assertExpected(valueList, 170749);
}
@Test
public void testNullRHS() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongFloat(100L, null)), 71);
}
@Test
public void testEmpty() throws Exception
{
// minimum size for empty data
assertExpected(Collections.emptyList(), 57);
}
@Test
public void testSingleNull() throws Exception
{
assertExpected(Arrays.asList(new SerializablePairLongFloat[]{null}), 58);
}
@Test
public void testMultipleNull() throws Exception
{
assertExpected(Arrays.asList(null, null, null, null), 59);
}
private ByteBuffer assertExpected(
List<SerializablePairLongFloat> expected,
int expectedCompressedSize
) throws IOException
{
SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer(
expected,
COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used"),
expectedCompressedSize
).asReadOnlyBuffer();
try (ComplexColumn compressedCol = createComplexColumn(compressedBuffer)
) {
for (int i = 0; i < expected.size(); i++) {
Assert.assertEquals(expected.get(i), compressedCol.getRowValue(i));
}
}
return compressedBuffer;
}
private ComplexColumn createComplexColumn(ByteBuffer byteBuffer)
{
ColumnBuilder builder = new ColumnBuilder();
int serializedSize = byteBuffer.remaining();
COMPRESSED_SERDE.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
final ComplexColumn col = (ComplexColumn) columnHolder.getColumn();
if (col instanceof SerializablePairLongFloatComplexColumn) {
Assert.assertEquals(serializedSize, col.getLength());
}
Assert.assertEquals("serializablePairLongFloat", col.getTypeName());
Assert.assertEquals(SerializablePairLongFloat.class, col.getClazz());
return col;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private static ByteBuffer serializeAllValuesToByteBuffer(
List<SerializablePairLongFloat> values,
GenericColumnSerializer serializer,
int expectedSize
) throws IOException
{
serializer.open();
final AtomicReference<SerializablePairLongFloat> reference = new AtomicReference<>(null);
ColumnValueSelector<SerializablePairLongFloat> valueSelector =
new SingleObjectColumnValueSelector<SerializablePairLongFloat>(
SerializablePairLongFloat.class
)
{
@Nullable
@Override
public SerializablePairLongFloat getObject()
{
return reference.get();
}
};
for (SerializablePairLongFloat selector : values) {
reference.set(selector);
serializer.serialize(valueSelector);
}
return serializeToByteBuffer(serializer, expectedSize);
}
private static ByteBuffer serializeToByteBuffer(
GenericColumnSerializer<SerializablePairLongFloat> serializer,
int expectedSize
) throws IOException
{
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
serializer.writeTo(channel, null);
ByteBuffer byteBuffer = ByteBuffer.allocate((int) channel.size()).order(ByteOrder.nativeOrder());
channel.readFully(0, byteBuffer);
byteBuffer.flip();
if (expectedSize > -1) {
Assert.assertEquals(expectedSize, serializer.getSerializedSize());
}
Assert.assertEquals(serializer.getSerializedSize(), byteBuffer.limit());
return byteBuffer;
}
}

View File

@ -0,0 +1,99 @@
/*
* 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;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Random;
public class SerializablePairLongFloatDeltaEncodedStagedSerdeTest
{
private static final SerializablePairLongFloatDeltaEncodedStagedSerde INTEGER_SERDE =
new SerializablePairLongFloatDeltaEncodedStagedSerde(0L, true);
private static final SerializablePairLongFloatDeltaEncodedStagedSerde LONG_SERDE =
new SerializablePairLongFloatDeltaEncodedStagedSerde(0L, false);
private final Random random = new Random(0);
@Test
public void testNull()
{
assertValueEquals(null, 0, INTEGER_SERDE);
}
@Test
public void testSimpleInteger()
{
assertValueEquals(new SerializablePairLongFloat(100L, 10F), 9, INTEGER_SERDE);
}
@Test
public void testNullRHSInteger()
{
assertValueEquals(new SerializablePairLongFloat(100L, null), 5, INTEGER_SERDE);
}
@Test
public void testLargeRHSInteger()
{
assertValueEquals(
new SerializablePairLongFloat(100L, random.nextFloat()),
9,
INTEGER_SERDE
);
}
@Test
public void testSimpleLong()
{
assertValueEquals(new SerializablePairLongFloat(100L, 10F), 13, LONG_SERDE);
}
@Test
public void testNullRHSLong()
{
assertValueEquals(new SerializablePairLongFloat(100L, null), 9, LONG_SERDE);
}
@Test
public void testLargeRHSLong()
{
assertValueEquals(
new SerializablePairLongFloat(100L, random.nextFloat()),
13,
LONG_SERDE
);
}
private static void assertValueEquals(
@Nullable SerializablePairLongFloat value,
int size,
SerializablePairLongFloatDeltaEncodedStagedSerde serde
)
{
byte[] bytes = serde.serialize(value);
Assert.assertEquals(size, bytes.length);
SerializablePairLongFloat deserialized = serde.deserialize(bytes);
Assert.assertEquals(value, deserialized);
}
}

View File

@ -0,0 +1,350 @@
/*
* 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;
import com.google.common.primitives.Ints;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.Random;
public class SerializablePairLongLongBufferStoreTest
{
private final Random random = new Random(0);
private static final int MIN_INTEGER = 100;
private static final long MIN_LONG = 0L;
private final SerializablePairLongLong[] integerRangeArr = new SerializablePairLongLong[]{
new SerializablePairLongLong((long) MIN_INTEGER, 10L),
new SerializablePairLongLong(101L, 20L),
new SerializablePairLongLong(102L, 30L),
};
private final SerializablePairLongLong[] longRangeArr = new SerializablePairLongLong[]{
new SerializablePairLongLong((long) MIN_LONG, 10L),
new SerializablePairLongLong(101L, 20L),
new SerializablePairLongLong(102L, 30L),
new SerializablePairLongLong((long) Integer.MAX_VALUE, 40L),
new SerializablePairLongLong(Long.MAX_VALUE, 50L),
};
private final SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
private SerializablePairLongLongBufferStore bufferStore;
@Before
public void setup() throws Exception
{
bufferStore = new SerializablePairLongLongBufferStore(
new SerializedStorage<>(
writeOutMedium.makeWriteOutBytes(),
new SerializablePairLongLongSimpleStagedSerde()
)
);
}
@Test
public void testIteratorSimple() throws Exception
{
for (SerializablePairLongLong value : integerRangeArr) {
bufferStore.store(value);
}
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(integerRangeArr[i], iterator.next());
i++;
}
}
@Test
public void testIteratorNull() throws Exception
{
bufferStore.store(null);
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertNull(iterator.next());
}
@Test
public void testIteratorIdempotentHasNext() throws Exception
{
bufferStore.store(integerRangeArr[0]);
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
// expect hasNext() to not modify state
Assert.assertTrue(iterator.hasNext());
}
@Test(expected = NoSuchElementException.class)
public void testIteratorEmptyThrows() throws Exception
{
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
iterator.next();
}
@Test
public void testIteratorEmptyHasNext() throws Exception
{
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testMinValueUsesInteger() throws Exception
{
for (SerializablePairLongLong value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(integerRangeArr[0].lhs.longValue(), columnHeader.getMinValue());
Assert.assertTrue(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesLong() throws Exception
{
for (SerializablePairLongLong value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(MIN_LONG, columnHeader.getMinValue());
Assert.assertFalse(columnHeader.isUseIntegerDeltas());
}
@Test
public void testMinValueUsesIntegerSerialization() throws Exception
{
for (SerializablePairLongLong value : integerRangeArr) {
bufferStore.store(value);
}
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongLongColumnHeader deserializedColumnhHeader =
(SerializablePairLongLongColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongLong.class);
Assert.assertEquals(MIN_INTEGER, deserializedColumnhHeader.getMinValue());
Assert.assertTrue(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testMinValueSerialization() throws Exception
{
for (SerializablePairLongLong value : longRangeArr) {
bufferStore.store(value);
}
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
columnHeader.transferTo(channel);
ByteBuffer byteBuffer = resourceHolder.get();
channel.writeTo(byteBuffer);
byteBuffer.flip();
SerializablePairLongLongColumnHeader deserializedColumnhHeader =
(SerializablePairLongLongColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongLong.class);
Assert.assertEquals(MIN_LONG, deserializedColumnhHeader.getMinValue());
Assert.assertFalse(deserializedColumnhHeader.isUseIntegerDeltas());
}
}
@Test
public void testLargeBuffer() throws Exception
{
SerializablePairLongLong value =
new SerializablePairLongLong(Long.MAX_VALUE, Long.MAX_VALUE);
bufferStore.store(value);
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
Assert.assertTrue(iterator.hasNext());
Assert.assertEquals(value, iterator.next());
Assert.assertFalse(iterator.hasNext());
}
@Test
public void testLargeValueCount() throws Exception
{
List<SerializablePairLongLong> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongLong(Integer.MAX_VALUE + (long) i, random.nextLong()));
}
assertBufferedValuesEqual(valueList);
}
@Test
public void testOverflowTransfer() throws Exception
{
bufferStore.store(new SerializablePairLongLong(Long.MIN_VALUE, 10L));
bufferStore.store(new SerializablePairLongLong(Long.MAX_VALUE, 10L));
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(94, transferredBuffer.getSerializedSize());
}
@Test
public void testNullOnlyTransfer() throws Exception
{
bufferStore.store(null);
bufferStore.store(null);
bufferStore.store(null);
SerializablePairLongLongColumnHeader columnHeader = (SerializablePairLongLongColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
Assert.assertEquals(59, transferredBuffer.getSerializedSize());
}
@Test
public void testTransferIntegerRange() throws Exception
{
for (SerializablePairLongLong value : integerRangeArr) {
bufferStore.store(value);
}
Assert.assertTrue(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(integerRangeArr);
}
@Test
public void testTransferLongRange() throws Exception
{
for (SerializablePairLongLong value : longRangeArr) {
bufferStore.store(value);
}
Assert.assertFalse(bufferStore.createColumnHeader().isUseIntegerDeltas());
assertTransferredValuesEqual(longRangeArr);
}
private void assertBufferedValuesEqual(List<SerializablePairLongLong> input) throws IOException
{
for (SerializablePairLongLong pairLongLong : input) {
bufferStore.store(pairLongLong);
}
IOIterator<SerializablePairLongLong> iterator = bufferStore.iterator();
int i = 0;
while (iterator.hasNext()) {
Assert.assertEquals(input.get(i), iterator.next());
i++;
}
Assert.assertEquals(
StringUtils.format("element count mismatch: expected %s, got %s", input.size(), i),
input.size(),
i
);
}
private void assertTransferredValuesEqual(SerializablePairLongLong[] input) throws IOException
{
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer =
bufferStore.transferToRowWriter(NativeClearedByteBufferProvider.INSTANCE, writeOutMedium);
HeapByteBufferWriteOutBytes resultChannel = new HeapByteBufferWriteOutBytes();
transferredBuffer.writeTo(resultChannel, null);
try (SerializablePairLongLongComplexColumn column = createComplexColumn(transferredBuffer, resultChannel)) {
for (int i = 0; i < input.length; i++) {
Assert.assertEquals(input[i], column.getRowValue(i));
}
}
}
private static SerializablePairLongLongComplexColumn createComplexColumn(
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer,
HeapByteBufferWriteOutBytes resultChannel
)
{
ByteBuffer byteBuffer = ByteBuffer.allocate(Ints.checkedCast(transferredBuffer.getSerializedSize()));
resultChannel.readFully(0, byteBuffer);
byteBuffer.flip();
SerializablePairLongLongComplexMetricSerde complexMetricSerde = new SerializablePairLongLongComplexMetricSerde();
ColumnBuilder builder = new ColumnBuilder();
complexMetricSerde.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
SerializablePairLongLongComplexColumn column = (SerializablePairLongLongComplexColumn) columnHolder.getColumn();
return column;
}
}

View File

@ -0,0 +1,238 @@
/*
* 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;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.GenericColumnSerializer;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ComplexColumn;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes;
import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicReference;
public class SerializablePairLongLongComplexMetricSerdeTest
{
static {
NullHandling.initializeForTests();
}
private static final SerializablePairLongLongComplexMetricSerde COMPRESSED_SERDE =
new SerializablePairLongLongComplexMetricSerde();
// want deterministic test input
private final Random random = new Random(0);
@Test
public void testSingle() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongLong(100L, 10L)), 78);
}
@Test
public void testLargeRHS() throws Exception
{
// single entry spans more than one block in underlying storage
assertExpected(ImmutableList.of(new SerializablePairLongLong(
100L,
random.nextLong()
)), 78);
}
@Test
public void testCompressable() throws Exception
{
int numLongs = 10;
List<SerializablePairLongLong> valueList = new ArrayList<>();
List<Long> longList = new ArrayList<>();
for (int i = 0; i < numLongs; i++) {
longList.add(random.nextLong());
}
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongLong(Integer.MAX_VALUE + (long) i, longList.get(i % numLongs)));
}
assertExpected(valueList, 80509);
}
@Test
public void testHighlyCompressable() throws Exception
{
List<SerializablePairLongLong> valueList = new ArrayList<>();
Long longValue = random.nextLong();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongLong(Integer.MAX_VALUE + (long) i, longValue));
}
assertExpected(valueList, 80274);
}
@Test
public void testRandom() throws Exception
{
List<SerializablePairLongLong> valueList = new ArrayList<>();
for (int i = 0; i < 10000; i++) {
valueList.add(new SerializablePairLongLong(random.nextLong(), random.nextLong()));
}
assertExpected(valueList, 210967);
}
@Test
public void testNullRHS() throws Exception
{
assertExpected(ImmutableList.of(new SerializablePairLongLong(100L, null)), 71);
}
@Test
public void testEmpty() throws Exception
{
// minimum size for empty data
assertExpected(Collections.emptyList(), 57);
}
@Test
public void testSingleNull() throws Exception
{
assertExpected(Arrays.asList(new SerializablePairLongLong[]{null}), 58);
}
@Test
public void testMultipleNull() throws Exception
{
assertExpected(Arrays.asList(null, null, null, null), 59);
}
private ByteBuffer assertExpected(
List<SerializablePairLongLong> expected,
int expectedCompressedSize
) throws IOException
{
SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium();
ByteBuffer compressedBuffer = serializeAllValuesToByteBuffer(
expected,
COMPRESSED_SERDE.getSerializer(writeOutMedium, "not-used"),
expectedCompressedSize
).asReadOnlyBuffer();
try (ComplexColumn compressedCol = createComplexColumn(compressedBuffer)
) {
for (int i = 0; i < expected.size(); i++) {
Assert.assertEquals(expected.get(i), compressedCol.getRowValue(i));
}
}
return compressedBuffer;
}
private ComplexColumn createComplexColumn(ByteBuffer byteBuffer)
{
ColumnBuilder builder = new ColumnBuilder();
int serializedSize = byteBuffer.remaining();
COMPRESSED_SERDE.deserializeColumn(byteBuffer, builder);
builder.setType(ValueType.COMPLEX);
ColumnHolder columnHolder = builder.build();
final ComplexColumn col = (ComplexColumn) columnHolder.getColumn();
if (col instanceof SerializablePairLongLongComplexColumn) {
Assert.assertEquals(serializedSize, col.getLength());
}
Assert.assertEquals("serializablePairLongLong", col.getTypeName());
Assert.assertEquals(SerializablePairLongLong.class, col.getClazz());
return col;
}
@SuppressWarnings({"unchecked", "rawtypes"})
private static ByteBuffer serializeAllValuesToByteBuffer(
List<SerializablePairLongLong> values,
GenericColumnSerializer serializer,
int expectedSize
) throws IOException
{
serializer.open();
final AtomicReference<SerializablePairLongLong> reference = new AtomicReference<>(null);
ColumnValueSelector<SerializablePairLongLong> valueSelector =
new SingleObjectColumnValueSelector<SerializablePairLongLong>(
SerializablePairLongLong.class
)
{
@Nullable
@Override
public SerializablePairLongLong getObject()
{
return reference.get();
}
};
for (SerializablePairLongLong selector : values) {
reference.set(selector);
serializer.serialize(valueSelector);
}
return serializeToByteBuffer(serializer, expectedSize);
}
private static ByteBuffer serializeToByteBuffer(
GenericColumnSerializer<SerializablePairLongLong> serializer,
int expectedSize
) throws IOException
{
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
serializer.writeTo(channel, null);
ByteBuffer byteBuffer = ByteBuffer.allocate((int) channel.size()).order(ByteOrder.nativeOrder());
channel.readFully(0, byteBuffer);
byteBuffer.flip();
if (expectedSize > -1) {
Assert.assertEquals(expectedSize, serializer.getSerializedSize());
}
Assert.assertEquals(serializer.getSerializedSize(), byteBuffer.limit());
return byteBuffer;
}
}

View File

@ -0,0 +1,99 @@
/*
* 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;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Random;
public class SerializablePairLongLongDeltaEncodedStagedSerdeTest
{
private static final SerializablePairLongLongDeltaEncodedStagedSerde INTEGER_SERDE =
new SerializablePairLongLongDeltaEncodedStagedSerde(0L, true);
private static final SerializablePairLongLongDeltaEncodedStagedSerde LONG_SERDE =
new SerializablePairLongLongDeltaEncodedStagedSerde(0L, false);
private final Random random = new Random(0);
@Test
public void testNull()
{
assertValueEquals(null, 0, INTEGER_SERDE);
}
@Test
public void testSimpleInteger()
{
assertValueEquals(new SerializablePairLongLong(100L, 10L), 13, INTEGER_SERDE);
}
@Test
public void testNullRHSInteger()
{
assertValueEquals(new SerializablePairLongLong(100L, null), 5, INTEGER_SERDE);
}
@Test
public void testLargeRHSInteger()
{
assertValueEquals(
new SerializablePairLongLong(100L, random.nextLong()),
13,
INTEGER_SERDE
);
}
@Test
public void testSimpleLong()
{
assertValueEquals(new SerializablePairLongLong(100L, 10L), 17, LONG_SERDE);
}
@Test
public void testNullRHSLong()
{
assertValueEquals(new SerializablePairLongLong(100L, null), 9, LONG_SERDE);
}
@Test
public void testLargeRHSLong()
{
assertValueEquals(
new SerializablePairLongLong(100L, random.nextLong()),
17,
LONG_SERDE
);
}
private static void assertValueEquals(
@Nullable SerializablePairLongLong value,
int size,
SerializablePairLongLongDeltaEncodedStagedSerde serde
)
{
byte[] bytes = serde.serialize(value);
Assert.assertEquals(size, bytes.length);
SerializablePairLongLong deserialized = serde.deserialize(bytes);
Assert.assertEquals(value, deserialized);
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.query.aggregation;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Random;
public class SerializablePairLongLongSimpleStagedSerdeTest
{
private static final SerializablePairLongLongSimpleStagedSerde SERDE =
new SerializablePairLongLongSimpleStagedSerde();
private final Random random = new Random(0);
@Test
public void testSimple()
{
assertValueEquals(new SerializablePairLongLong(Long.MAX_VALUE, 10L), 17);
}
@Test
public void testNull()
{
assertValueEquals(null, 0);
}
@Test
public void testNullString()
{
assertValueEquals(new SerializablePairLongLong(Long.MAX_VALUE, null), 9);
}
@Test
public void testLargeRHS()
{
assertValueEquals(
new SerializablePairLongLong(Long.MAX_VALUE, random.nextLong()),
17
);
}
private static void assertValueEquals(@Nullable SerializablePairLongLong value, int size)
{
byte[] bytes = SERDE.serialize(value);
Assert.assertEquals(size, bytes.length);
SerializablePairLongLong deserialized = SERDE.deserialize(bytes);
Assert.assertEquals(value, deserialized);
}
}

View File

@ -69,7 +69,7 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore = new SerializablePairLongStringBufferStore(
new SerializedStorage<>(
writeOutMedium.makeWriteOutBytes(),
SerializablePairLongStringColumnSerializer.STAGED_SERDE
new SerializablePairLongStringSimpleStagedSerde()
));
}
@ -139,7 +139,7 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(value);
}
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(integerRangeArr[0].lhs.longValue(), columnHeader.getMinValue());
Assert.assertTrue(columnHeader.isUseIntegerDeltas());
}
@ -151,7 +151,7 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(value);
}
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(MIN_LONG, columnHeader.getMinValue());
Assert.assertFalse(columnHeader.isUseIntegerDeltas());
}
@ -163,7 +163,7 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(value);
}
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
@ -174,7 +174,7 @@ public class SerializablePairLongStringBufferStoreTest
byteBuffer.flip();
SerializablePairLongStringColumnHeader deserializedColumnhHeader =
SerializablePairLongStringColumnHeader.fromBuffer(byteBuffer);
(SerializablePairLongStringColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongString.class);
Assert.assertEquals(MIN_INTEGER, deserializedColumnhHeader.getMinValue());
Assert.assertTrue(deserializedColumnhHeader.isUseIntegerDeltas());
}
@ -188,7 +188,7 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(value);
}
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes();
try (ResourceHolder<ByteBuffer> resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) {
@ -200,7 +200,7 @@ public class SerializablePairLongStringBufferStoreTest
byteBuffer.flip();
SerializablePairLongStringColumnHeader deserializedColumnhHeader =
SerializablePairLongStringColumnHeader.fromBuffer(byteBuffer);
(SerializablePairLongStringColumnHeader) AbstractSerializablePairLongObjectColumnHeader.fromBuffer(byteBuffer, SerializablePairLongString.class);
Assert.assertEquals(MIN_LONG, deserializedColumnhHeader.getMinValue());
Assert.assertFalse(deserializedColumnhHeader.isUseIntegerDeltas());
}
@ -271,11 +271,11 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(new SerializablePairLongString(Long.MIN_VALUE, "fuu"));
bufferStore.store(new SerializablePairLongString(Long.MAX_VALUE, "fuu"));
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
@ -292,11 +292,11 @@ public class SerializablePairLongStringBufferStoreTest
bufferStore.store(null);
SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader();
SerializablePairLongStringColumnHeader columnHeader = (SerializablePairLongStringColumnHeader) bufferStore.createColumnHeader();
Assert.assertEquals(0, columnHeader.getMinValue());
SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter(
NativeClearedByteBufferProvider.INSTANCE,
writeOutMedium
);
@ -351,7 +351,7 @@ public class SerializablePairLongStringBufferStoreTest
private void assertTransferredValuesEqual(SerializablePairLongString[] input) throws IOException
{
SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer =
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer =
bufferStore.transferToRowWriter(NativeClearedByteBufferProvider.INSTANCE, writeOutMedium);
HeapByteBufferWriteOutBytes resultChannel = new HeapByteBufferWriteOutBytes();
@ -365,7 +365,7 @@ public class SerializablePairLongStringBufferStoreTest
}
private static SerializablePairLongStringComplexColumn createComplexColumn(
SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer,
AbstractSerializablePairLongObjectBufferStore.TransferredBuffer transferredBuffer,
HeapByteBufferWriteOutBytes resultChannel
)
{

View File

@ -19,17 +19,20 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.query.aggregation.TestDoubleColumnSelectorImpl;
import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock;
import org.junit.Assert;
@ -52,11 +55,11 @@ public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
private double[] doubleValues = {1.1d, 2.7d, 3.5d, 1.3d};
private long[] times = {12, 10, 5344, 7899999};
private long[] customTimes = {2, 1, 3, 4};
private SerializablePair[] pairs = {
new SerializablePair<>(1467225096L, 134.3d),
new SerializablePair<>(23163L, 1232.212d),
new SerializablePair<>(742L, 18d),
new SerializablePair<>(111111L, 233.5232d)
private SerializablePairLongDouble[] pairs = {
new SerializablePairLongDouble(1467225096L, 134.3d),
new SerializablePairLongDouble(23163L, 1232.212d),
new SerializablePairLongDouble(742L, 18d),
new SerializablePairLongDouble(111111L, 233.5232d)
};
@Before
@ -73,6 +76,10 @@ public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("customTime")).andReturn(customTimeSelector);
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector);
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector);
EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly"))
.andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.DOUBLE));
EasyMock.expect(colSelectorFactory.getColumnCapabilities("billy")).andReturn(null);
EasyMock.replay(colSelectorFactory);
}
@ -158,16 +165,16 @@ public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
@Test
public void testCombine()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621);
SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4);
SerializablePairLongDouble pair1 = new SerializablePairLongDouble(1467225000L, 3.621);
SerializablePairLongDouble pair2 = new SerializablePairLongDouble(1467240000L, 785.4);
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);
SerializablePairLongDouble pair1 = new SerializablePairLongDouble(1467225000L, 3.621);
SerializablePairLongDouble pair2 = new SerializablePairLongDouble(1467240000L, 785.4);
Comparator comparator = doubleFirstAggFactory.getComparator();
Assert.assertEquals(-1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
@ -178,8 +185,8 @@ public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
SerializablePairLongDouble pair1 = new SerializablePairLongDouble(1467225000L, 3.621);
SerializablePairLongDouble pair2 = new SerializablePairLongDouble(1467240000L, null);
Comparator comparator = doubleFirstAggFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
@ -241,6 +248,28 @@ public class DoubleFirstAggregationTest extends InitializedNullHandlingTest
Assert.assertArrayEquals(doubleFirstAggFactory.getCacheKey(), deserialized.getCacheKey());
}
@Test
public void testDoubleFirstAggregateCombiner()
{
TestObjectColumnSelector columnSelector = new TestObjectColumnSelector<>(pairs);
AggregateCombiner doubleFirstAggregateCombiner = combiningAggFactory.makeAggregateCombiner();
doubleFirstAggregateCombiner.reset(columnSelector);
Assert.assertEquals(pairs[0], doubleFirstAggregateCombiner.getObject());
columnSelector.increment();
doubleFirstAggregateCombiner.fold(columnSelector);
Assert.assertEquals(pairs[1], doubleFirstAggregateCombiner.getObject());
columnSelector.increment();
doubleFirstAggregateCombiner.fold(columnSelector);
Assert.assertEquals(pairs[2], doubleFirstAggregateCombiner.getObject());
doubleFirstAggregateCombiner.reset(columnSelector);
Assert.assertEquals(pairs[2], doubleFirstAggregateCombiner.getObject());
}
private void aggregate(
Aggregator agg
)

View File

@ -19,14 +19,13 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.aggregation.SerializablePairLongDouble;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.NoFilterVectorOffset;
@ -48,23 +47,29 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
{
private static final double EPSILON = 1e-5;
private static final double[] VALUES = new double[]{7.8d, 11, 23.67, 60};
private static final boolean[] NULLS = new boolean[]{false, false, true, false};
private long[] times = {2436, 6879, 7888, 8224};
private static final long[] LONG_VALUES = new long[]{1L, 2L, 3L, 4L};
private static final float[] FLOAT_VALUES = new float[]{1.0f, 2.0f, 3.0f, 4.0f};
private static final double[] DOUBLE_VALUES = new double[]{1.0, 2.0, 3.0, 4.0};
private static final String NAME = "NAME";
private static final String FIELD_NAME = "FIELD_NAME";
private static final String FIELD_NAME_LONG = "LONG_NAME";
private static final String TIME_COL = "__time";
private final long[] times = {2345001L, 2345100L, 2345200L, 2345300L};
private final SerializablePairLongDouble[] pairs = {
new SerializablePairLongDouble(2345001L, 1D),
new SerializablePairLongDouble(2345100L, 2D),
new SerializablePairLongDouble(2345200L, 3D),
new SerializablePairLongDouble(2345300L, 4D)
};
private VectorValueSelector selector;
private VectorObjectSelector selector;
private BaseLongVectorValueSelector timeSelector;
private ByteBuffer buf;
private DoubleFirstVectorAggregator target;
private DoubleFirstAggregatorFactory doubleFirstAggregatorFactory;
private VectorColumnSelectorFactory selectorFactory;
private VectorValueSelector nonLongValueSelector;
@Before
public void setup()
@ -86,39 +91,80 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
@Override
public boolean[] getNullVector()
{
return NULLS;
return null;
}
};
selector = new BaseDoubleVectorValueSelector(new NoFilterVectorOffset(VALUES.length, 0, VALUES.length)
selector = new VectorObjectSelector()
{
@Override
public Object[] getObjectVector()
{
return pairs;
}
})
@Override
public int getMaxVectorSize()
{
return 4;
}
@Override
public int getCurrentVectorSize()
{
return 0;
}
};
nonLongValueSelector = new BaseLongVectorValueSelector(new NoFilterVectorOffset(
LONG_VALUES.length,
0,
LONG_VALUES.length
))
{
@Override
public long[] getLongVector()
{
return LONG_VALUES;
}
@Override
public float[] getFloatVector()
{
return FLOAT_VALUES;
}
@Override
public double[] getDoubleVector()
{
return VALUES;
return DOUBLE_VALUES;
}
@Nullable
@Override
public boolean[] getNullVector()
{
if (!NullHandling.replaceWithDefault()) {
return NULLS;
}
return null;
}
@Override
public int getMaxVectorSize()
{
return 4;
}
@Override
public int getCurrentVectorSize()
{
return 4;
}
};
target = new DoubleFirstVectorAggregator(timeSelector, selector);
clearBufferForPositions(0, 0);
selectorFactory = new VectorColumnSelectorFactory()
{
@Override
public ReadableVectorInspector getReadableVectorInspector()
{
return null;
return new NoFilterVectorOffset(VALUES.length, 0, VALUES.length);
}
@Override
@ -138,17 +184,21 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
{
if (TIME_COL.equals(column)) {
return timeSelector;
} else if (FIELD_NAME.equals(column)) {
return selector;
} else {
return null;
} else if (FIELD_NAME_LONG.equals(column)) {
return nonLongValueSelector;
}
return null;
}
@Override
public VectorObjectSelector makeObjectSelector(String column)
{
return null;
if (FIELD_NAME.equals(column)) {
return selector;
} else {
return null;
}
}
@Nullable
@ -157,11 +207,16 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
{
if (FIELD_NAME.equals(column)) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE);
} else if (FIELD_NAME_LONG.equals(column)) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG);
}
return null;
}
};
target = new DoubleFirstVectorAggregator(timeSelector, selector);
clearBufferForPositions(0, 0);
doubleFirstAggregatorFactory = new DoubleFirstAggregatorFactory(NAME, FIELD_NAME, TIME_COL);
}
@ -185,19 +240,19 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
@Test
public void aggregate()
{
target.aggregate(buf, 0, 0, VALUES.length);
target.aggregate(buf, 0, 0, pairs.length);
Pair<Long, Double> result = (Pair<Long, Double>) target.get(buf, 0);
Assert.assertEquals(times[0], result.lhs.longValue());
Assert.assertEquals(VALUES[0], result.rhs, EPSILON);
Assert.assertEquals(pairs[0].lhs.longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[0].rhs, result.rhs, EPSILON);
}
@Test
public void aggregateWithNulls()
{
target.aggregate(buf, 0, 0, VALUES.length);
target.aggregate(buf, 0, 0, pairs.length);
Pair<Long, Double> result = (Pair<Long, Double>) target.get(buf, 0);
Assert.assertEquals(times[0], result.lhs.longValue());
Assert.assertEquals(VALUES[0], result.rhs, EPSILON);
Assert.assertEquals(pairs[0].lhs.longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[0].rhs, result.rhs, EPSILON);
}
@Test
@ -209,12 +264,8 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
target.aggregate(buf, 3, positions, null, positionOffset);
for (int i = 0; i < positions.length; i++) {
Pair<Long, Double> result = (Pair<Long, Double>) target.get(buf, positions[i] + positionOffset);
Assert.assertEquals(times[i], result.lhs.longValue());
if (!NullHandling.replaceWithDefault() && NULLS[i]) {
Assert.assertNull(result.rhs);
} else {
Assert.assertEquals(VALUES[i], result.rhs, EPSILON);
}
Assert.assertEquals(pairs[i].getLhs().longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[i].rhs, result.rhs, EPSILON);
}
}
@ -222,18 +273,15 @@ public class DoubleFirstVectorAggregationTest extends InitializedNullHandlingTes
public void aggregateBatchWithRows()
{
int[] positions = new int[]{0, 43, 70};
int[] rows = new int[]{3, 2, 0};
int[] rows = new int[]{3, 0, 2};
int positionOffset = 2;
clearBufferForPositions(positionOffset, positions);
target.aggregate(buf, 3, positions, rows, positionOffset);
for (int i = 0; i < positions.length; i++) {
Pair<Long, Double> result = (Pair<Long, Double>) target.get(buf, positions[i] + positionOffset);
Assert.assertEquals(times[rows[i]], result.lhs.longValue());
if (!NullHandling.replaceWithDefault() && NULLS[rows[i]]) {
Assert.assertNull(result.rhs);
} else {
Assert.assertEquals(VALUES[rows[i]], result.rhs, EPSILON);
}
Assert.assertEquals(pairs[rows[i]].lhs.longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[rows[i]].rhs, result.rhs, EPSILON);
}
}

View File

@ -19,17 +19,20 @@
package org.apache.druid.query.aggregation.first;
import org.apache.druid.collections.SerializablePair;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.aggregation.AggregateCombiner;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.query.aggregation.TestFloatColumnSelector;
import org.apache.druid.query.aggregation.TestLongColumnSelector;
import org.apache.druid.query.aggregation.TestObjectColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.easymock.EasyMock;
import org.junit.Assert;
@ -52,11 +55,11 @@ public class FloatFirstAggregationTest extends InitializedNullHandlingTest
private float[] floats = {1.1f, 2.7f, 3.5f, 1.3f};
private long[] times = {12, 10, 5344, 7899999};
private long[] customTimes = {2, 1, 3, 4};
private SerializablePair[] pairs = {
new SerializablePair<>(1467225096L, 134.3f),
new SerializablePair<>(23163L, 1232.212f),
new SerializablePair<>(742L, 18f),
new SerializablePair<>(111111L, 233.5232f)
private SerializablePairLongFloat[] pairs = {
new SerializablePairLongFloat(1467225096L, 134.3f),
new SerializablePairLongFloat(23163L, 1232.212f),
new SerializablePairLongFloat(742L, 18f),
new SerializablePairLongFloat(111111L, 233.5232f)
};
@Before
@ -73,6 +76,10 @@ public class FloatFirstAggregationTest extends InitializedNullHandlingTest
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("customTime")).andReturn(customTimeSelector);
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector).atLeastOnce();
EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector).atLeastOnce();
EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly"))
.andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.FLOAT));
EasyMock.expect(colSelectorFactory.getColumnCapabilities("billy")).andReturn(null);
EasyMock.replay(colSelectorFactory);
}
@ -158,16 +165,16 @@ public class FloatFirstAggregationTest extends InitializedNullHandlingTest
@Test
public void testCombine()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, 785.4f);
SerializablePairLongFloat pair1 = new SerializablePairLongFloat(1467225000L, 3.621f);
SerializablePairLongFloat pair2 = new SerializablePairLongFloat(1467240000L, 785.4f);
Assert.assertEquals(pair1, floatFirstAggregatorFactory.combine(pair1, pair2));
}
@Test
public void testComparatorWithNulls()
{
SerializablePair pair1 = new SerializablePair<>(1467225000L, 3.621f);
SerializablePair pair2 = new SerializablePair<>(1467240000L, null);
SerializablePairLongFloat pair1 = new SerializablePairLongFloat(1467225000L, 3.621f);
SerializablePairLongFloat pair2 = new SerializablePairLongFloat(1467240000L, null);
Comparator comparator = floatFirstAggregatorFactory.getComparator();
Assert.assertEquals(1, comparator.compare(pair1, pair2));
Assert.assertEquals(0, comparator.compare(pair1, pair1));
@ -228,6 +235,28 @@ public class FloatFirstAggregationTest extends InitializedNullHandlingTest
Assert.assertArrayEquals(floatFirstAggregatorFactory.getCacheKey(), deserialized.getCacheKey());
}
@Test
public void testFloatFirstAggregateCombiner()
{
TestObjectColumnSelector columnSelector = new TestObjectColumnSelector<>(pairs);
AggregateCombiner floatFirstAggregateCombiner = combiningAggFactory.makeAggregateCombiner();
floatFirstAggregateCombiner.reset(columnSelector);
Assert.assertEquals(pairs[0], floatFirstAggregateCombiner.getObject());
columnSelector.increment();
floatFirstAggregateCombiner.fold(columnSelector);
Assert.assertEquals(pairs[1], floatFirstAggregateCombiner.getObject());
columnSelector.increment();
floatFirstAggregateCombiner.fold(columnSelector);
Assert.assertEquals(pairs[2], floatFirstAggregateCombiner.getObject());
floatFirstAggregateCombiner.reset(columnSelector);
Assert.assertEquals(pairs[2], floatFirstAggregateCombiner.getObject());
}
private void aggregate(
Aggregator agg
)

View File

@ -21,12 +21,12 @@ package org.apache.druid.query.aggregation.first;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.query.aggregation.SerializablePairLongFloat;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.vector.BaseFloatVectorValueSelector;
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.NoFilterVectorOffset;
@ -48,24 +48,32 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
{
private static final double EPSILON = 1e-5;
private static final float[] VALUES = new float[]{7.2f, 15.6f, 2.1f, 150.0f};
private static final boolean[] NULLS = new boolean[]{false, false, true, false};
private long[] times = {2436, 6879, 7888, 8224};
private static final long[] LONG_VALUES = new long[]{1L, 2L, 3L, 4L};
private static final float[] FLOAT_VALUES = new float[]{1.0f, 2.0f, 3.0f, 4.0f};
private static final double[] DOUBLE_VALUES = new double[]{1.0, 2.0, 3.0, 4.0};
private static final boolean[] NULLS = new boolean[]{false, false, false, false};
private static final String NAME = "NAME";
private static final String FIELD_NAME = "FIELD_NAME";
private static final String FIELD_NAME_LONG = "LONG_NAME";
private static final String TIME_COL = "__time";
private final long[] times = {2345001L, 2345100L, 2345200L, 2345300L};
private final SerializablePairLongFloat[] pairs = {
new SerializablePairLongFloat(2345001L, 1.2F),
new SerializablePairLongFloat(2345100L, 2.2F),
new SerializablePairLongFloat(2345200L, 3.2F),
new SerializablePairLongFloat(2345300L, 4.2F)
};
private VectorValueSelector selector;
private VectorObjectSelector selector;
private BaseLongVectorValueSelector timeSelector;
private ByteBuffer buf;
private FloatFirstVectorAggregator target;
private FloatFirstAggregatorFactory floatFirstAggregatorFactory;
private VectorColumnSelectorFactory selectorFactory;
private VectorValueSelector nonFloatValueSelector;
@Before
public void setup()
@ -87,41 +95,80 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
@Override
public boolean[] getNullVector()
{
return NULLS;
return null;
}
};
selector = new BaseFloatVectorValueSelector(new NoFilterVectorOffset(VALUES.length, 0, VALUES.length)
selector = new VectorObjectSelector()
{
@Override
public Object[] getObjectVector()
{
return pairs;
}
})
@Override
public int getMaxVectorSize()
{
return 4;
}
@Override
public int getCurrentVectorSize()
{
return 0;
}
};
nonFloatValueSelector = new BaseLongVectorValueSelector(new NoFilterVectorOffset(
LONG_VALUES.length,
0,
LONG_VALUES.length
))
{
@Override
public long[] getLongVector()
{
return LONG_VALUES;
}
@Override
public float[] getFloatVector()
{
return VALUES;
return FLOAT_VALUES;
}
@Override
public double[] getDoubleVector()
{
return DOUBLE_VALUES;
}
@Nullable
@Override
public boolean[] getNullVector()
{
if (!NullHandling.replaceWithDefault()) {
return NULLS;
}
return null;
return NULLS;
}
@Override
public int getMaxVectorSize()
{
return 4;
}
@Override
public int getCurrentVectorSize()
{
return 4;
}
};
target = new FloatFirstVectorAggregator(timeSelector, selector);
clearBufferForPositions(0, 0);
selectorFactory = new VectorColumnSelectorFactory()
{
@Override
public ReadableVectorInspector getReadableVectorInspector()
{
return null;
return new NoFilterVectorOffset(VALUES.length, 0, VALUES.length);
}
@Override
@ -142,7 +189,7 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
if (TIME_COL.equals(column)) {
return timeSelector;
} else if (FIELD_NAME.equals(column)) {
return selector;
return nonFloatValueSelector;
} else {
return null;
}
@ -151,7 +198,11 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
@Override
public VectorObjectSelector makeObjectSelector(String column)
{
return null;
if (FIELD_NAME.equals(column)) {
return selector;
} else {
return null;
}
}
@Nullable
@ -160,10 +211,16 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
{
if (FIELD_NAME.equals(column)) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT);
} else if (FIELD_NAME_LONG.equals(column)) {
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG);
}
return null;
}
};
target = new FloatFirstVectorAggregator(timeSelector, selector);
clearBufferForPositions(0, 0);
floatFirstAggregatorFactory = new FloatFirstAggregatorFactory(NAME, FIELD_NAME, TIME_COL);
}
@ -191,8 +248,8 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
target.init(buf, 0);
target.aggregate(buf, 0, 0, VALUES.length);
Pair<Long, Float> result = (Pair<Long, Float>) target.get(buf, 0);
Assert.assertEquals(times[0], result.lhs.longValue());
Assert.assertEquals(VALUES[0], result.rhs, EPSILON);
Assert.assertEquals(pairs[0].lhs.longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[0].rhs, result.rhs, EPSILON);
}
@Test
@ -200,8 +257,8 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
{
target.aggregate(buf, 0, 0, VALUES.length);
Pair<Long, Float> result = (Pair<Long, Float>) target.get(buf, 0);
Assert.assertEquals(times[0], result.lhs.longValue());
Assert.assertEquals(VALUES[0], result.rhs, EPSILON);
Assert.assertEquals(pairs[0].lhs.longValue(), result.lhs.longValue());
Assert.assertEquals(pairs[0].rhs, result.rhs, EPSILON);
}
@Test
@ -213,11 +270,11 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
target.aggregate(buf, 3, positions, null, positionOffset);
for (int i = 0; i < positions.length; i++) {
Pair<Long, Float> result = (Pair<Long, Float>) target.get(buf, positions[i] + positionOffset);
Assert.assertEquals(times[i], result.lhs.longValue());
Assert.assertEquals(pairs[i].getLhs().longValue(), result.lhs.longValue());
if (!NullHandling.replaceWithDefault() && NULLS[i]) {
Assert.assertNull(result.rhs);
} else {
Assert.assertEquals(VALUES[i], result.rhs, EPSILON);
Assert.assertEquals(pairs[i].rhs, result.rhs, EPSILON);
}
}
}
@ -236,7 +293,7 @@ public class FloatFirstVectorAggregationTest extends InitializedNullHandlingTest
if (!NullHandling.replaceWithDefault() && NULLS[rows[i]]) {
Assert.assertNull(result.rhs);
} else {
Assert.assertEquals(VALUES[rows[i]], result.rhs, EPSILON);
Assert.assertEquals(pairs[rows[i]].rhs, result.rhs, EPSILON);
}
}
}

Some files were not shown because too many files have changed in this diff Show More