Frame writers: Coerce numeric and array types in certain cases. (#16994)

This patch adds "TypeCastSelectors", which is used when writing frames to
perform two coercions:

- When a numeric type is desired and the underlying type is non-numeric or
  unknown, the underlying selector is wrapped, "getObject" is called and the
  result is coerced using "ExprEval.ofType". This differs from the prior
  behavior where the primitive methods like "getLong", "getDouble", etc, would
  be called directly. This fixes an issue where a column would be read as
  all-zeroes when its SQL type is numeric and its physical type is string, which
  can happen when evolving a column's type from string to number.

-  When an array type is desired, the underlying selector is wrapped,
   "getObject" is called, and the result is coerced to Object[]. This coercion
   replaces some earlier logic from #15917.
This commit is contained in:
Gian Merlino 2024-09-05 17:20:00 -07:00 committed by GitHub
parent c49dc83b22
commit 175636b28f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 835 additions and 117 deletions

View File

@ -109,7 +109,7 @@ public class GroupByPostShuffleFrameProcessor implements FrameProcessor<Object>
RowBasedGrouperHelper.createResultRowBasedColumnSelectorFactory(
query,
() -> outputRow,
RowSignature.Finalization.YES
GroupByQueryKit.isFinalize(query) ? RowSignature.Finalization.YES : RowSignature.Finalization.NO
)
);
}

View File

@ -4,7 +4,7 @@
"expectedResults": [
{
"__time": 1672058096000,
"double_col": 0.0
"double_col": null
}
]
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.frame.field;
import org.apache.druid.frame.key.RowKey;
import org.apache.druid.frame.write.RowBasedFrameWriterFactory;
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
import org.apache.druid.frame.write.cast.TypeCastSelectors;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.segment.ColumnSelectorFactory;
@ -101,7 +102,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.LONG);
return LongFieldWriter.forPrimitive(selector);
}
@ -110,7 +112,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.FLOAT);
return FloatFieldWriter.forPrimitive(selector);
}
@ -119,7 +122,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.DOUBLE);
return DoubleFieldWriter.forPrimitive(selector);
}
@ -139,7 +143,8 @@ public class FieldWriters
final boolean removeNullBytes
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.STRING_ARRAY);
return new StringArrayFieldWriter(selector, removeNullBytes);
}
@ -148,7 +153,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.LONG_ARRAY);
return NumericArrayFieldWriter.getLongArrayFieldWriter(selector);
}
@ -157,7 +163,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.FLOAT_ARRAY);
return NumericArrayFieldWriter.getFloatArrayFieldWriter(selector);
}
@ -166,7 +173,8 @@ public class FieldWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.DOUBLE_ARRAY);
return NumericArrayFieldWriter.getDoubleArrayFieldWriter(selector);
}
@ -185,7 +193,8 @@ public class FieldWriters
throw new ISE("No serde for complexTypeName[%s], cannot write column [%s]", columnTypeName, columnName);
}
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.ofComplex(columnTypeName));
return new ComplexFieldWriter(serde, selector);
}
}

View File

@ -39,7 +39,7 @@ import javax.annotation.Nullable;
*
* @param <ElementType> Type of the individual array elements
*/
public abstract class NumericArrayFieldSelector<ElementType extends Number> implements ColumnValueSelector
public abstract class NumericArrayFieldSelector<ElementType extends Number> implements ColumnValueSelector<Object[]>
{
/**
* Memory containing the serialized values of the array
@ -81,15 +81,15 @@ public abstract class NumericArrayFieldSelector<ElementType extends Number> impl
@Nullable
@Override
public Object getObject()
public Object[] getObject()
{
return computeCurrentArray();
}
@Override
public Class classOfObject()
public Class<Object[]> classOfObject()
{
return Object.class;
return Object[].class;
}
@Override
@ -131,7 +131,7 @@ public abstract class NumericArrayFieldSelector<ElementType extends Number> impl
public abstract int getIndividualFieldSize();
@Nullable
private Number[] computeCurrentArray()
private Object[] computeCurrentArray()
{
final long fieldPosition = fieldPointer.position();
final long fieldLength = fieldPointer.length();

View File

@ -21,12 +21,10 @@ package org.apache.druid.frame.field;
import org.apache.datasketches.memory.WritableMemory;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import javax.annotation.Nullable;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
/**
@ -128,7 +126,7 @@ public class NumericArrayFieldWriter implements FieldWriter
@Override
public long writeTo(WritableMemory memory, long position, long maxSize)
{
Object row = selector.getObject();
final Object[] row = (Object[]) selector.getObject();
if (row == null) {
int requiredSize = Byte.BYTES;
if (requiredSize > maxSize) {
@ -137,18 +135,6 @@ public class NumericArrayFieldWriter implements FieldWriter
memory.putByte(position, NULL_ROW);
return requiredSize;
} else {
List<? extends Number> list = FrameWriterUtils.getNumericArrayFromObject(row);
if (list == null) {
int requiredSize = Byte.BYTES;
if (requiredSize > maxSize) {
return -1;
}
memory.putByte(position, NULL_ROW);
return requiredSize;
}
// Create a columnValueSelector to write the individual elements re-using the NumericFieldWriter
AtomicInteger index = new AtomicInteger(0);
ColumnValueSelector<Number> columnValueSelector = new ColumnValueSelector<Number>()
@ -199,7 +185,7 @@ public class NumericArrayFieldWriter implements FieldWriter
@Override
public Number getObject()
{
return list.get(index.get());
return (Number) row[index.get()];
}
@Override
@ -215,7 +201,7 @@ public class NumericArrayFieldWriter implements FieldWriter
// Next [(1 + Numeric Size) x Number of elements of array] bytes are reserved for the elements of the array and
// their null markers
// Last byte is reserved for array termination
int requiredSize = Byte.BYTES + (writer.getNumericSizeBytes() + Byte.BYTES) * list.size() + Byte.BYTES;
int requiredSize = Byte.BYTES + (writer.getNumericSizeBytes() + Byte.BYTES) * row.length + Byte.BYTES;
if (requiredSize > maxSize) {
return -1;
@ -225,7 +211,7 @@ public class NumericArrayFieldWriter implements FieldWriter
memory.putByte(position + offset, NON_NULL_ROW);
offset += Byte.BYTES;
for (; index.get() < list.size(); index.incrementAndGet()) {
for (; index.get() < row.length; index.incrementAndGet()) {
writer.writeTo(
memory,
position + offset,

View File

@ -144,60 +144,16 @@ public class FrameWriterUtils
@SuppressWarnings("rawtypes") final BaseObjectColumnValueSelector selector
)
{
Object row = selector.getObject();
final Object[] row = (Object[]) selector.getObject();
if (row == null) {
return null;
} else if (row instanceof String) {
return Collections.singletonList(getUtf8ByteBufferFromString((String) row));
}
final List<ByteBuffer> retVal = new ArrayList<>();
if (row instanceof List) {
for (int i = 0; i < ((List<?>) row).size(); i++) {
retVal.add(getUtf8ByteBufferFromString(((List<String>) row).get(i)));
}
} else if (row instanceof Object[]) {
for (Object value : (Object[]) row) {
} else {
final List<ByteBuffer> retVal = new ArrayList<>();
for (Object value : row) {
retVal.add(getUtf8ByteBufferFromString((String) value));
}
} else {
throw new ISE("Unexpected type %s found", row.getClass().getName());
return retVal;
}
return retVal;
}
/**
* Retrieves a numeric list from a Java object, given that the object is an instance of something that can be returned
* from {@link ColumnValueSelector#getObject()} of valid numeric array selectors representations
*
* While {@link BaseObjectColumnValueSelector} specifies that only instances of {@code Object[]} can be returned from
* the numeric array selectors, this method also handles a few more cases which can be encountered if the selector is
* directly implemented on top of the group by stuff
*/
@Nullable
public static List<? extends Number> getNumericArrayFromObject(Object row)
{
if (row == null) {
return null;
} else if (row instanceof Number) {
return Collections.singletonList((Number) row);
}
final List<Number> retVal = new ArrayList<>();
if (row instanceof List) {
for (int i = 0; i < ((List<?>) row).size(); i++) {
retVal.add((Number) ((List<?>) row).get(i));
}
} else if (row instanceof Object[]) {
for (Object value : (Object[]) row) {
retVal.add((Number) value);
}
} else {
throw new ISE("Unexpected type %s found", row.getClass().getName());
}
return retVal;
}
/**
@ -275,6 +231,7 @@ public class FrameWriterUtils
* Whenever "allowNullBytes" is true, "removeNullBytes" must be false. Use the methods {@link #copyByteBufferToMemoryAllowingNullBytes}
* and {@link #copyByteBufferToMemoryDisallowingNullBytes} to copy between the memory
* <p>
*
* @throws InvalidNullByteException if "allowNullBytes" and "removeNullBytes" is false and a null byte is encountered
*/
private static void copyByteBufferToMemory(

View File

@ -0,0 +1,101 @@
/*
* 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.frame.write.cast;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.RowIdSupplier;
import javax.annotation.Nullable;
/**
* Wraps a {@link ColumnValueSelector}, calls {@link ColumnValueSelector#getObject()}, interprets that value using
* {@link ExprEval#ofType}, and casts it using {@link ExprEval#castTo}.
*/
public class ObjectToArrayColumnValueSelector implements ColumnValueSelector<Object[]>
{
private final ColumnValueSelector<?> selector;
@Nullable
private final ExpressionType desiredType;
@Nullable
private final RowIdSupplier rowIdSupplier;
public ObjectToArrayColumnValueSelector(
final ColumnValueSelector<?> selector,
final ExpressionType desiredType,
@Nullable final RowIdSupplier rowIdSupplier
)
{
this.selector = selector;
this.desiredType = desiredType;
this.rowIdSupplier = rowIdSupplier;
if (!desiredType.isArray() || desiredType.getElementType() == null) {
throw DruidException.defensive("Expected array with nonnull element type, got[%s]", desiredType);
}
}
@Override
public double getDouble()
{
throw DruidException.defensive("Unexpected call to getDouble on array selector");
}
@Override
public float getFloat()
{
throw DruidException.defensive("Unexpected call to getFloat on array selector");
}
@Override
public long getLong()
{
throw DruidException.defensive("Unexpected call to getLong on array selector");
}
@Override
public boolean isNull()
{
throw DruidException.defensive("Unexpected call to isNull on array selector");
}
@Nullable
@Override
public Object[] getObject()
{
return (Object[]) TypeCastSelectors.bestEffortCoerce(selector.getObject(), desiredType);
}
@Override
public Class<Object[]> classOfObject()
{
return Object[].class;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
inspector.visit("rowIdSupplier", rowIdSupplier);
}
}

View File

@ -0,0 +1,135 @@
/*
* 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.frame.write.cast;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.RowIdSupplier;
import javax.annotation.Nullable;
/**
* Wraps a {@link ColumnValueSelector}, calls {@link ColumnValueSelector#getObject()} and provides primitive numeric
* accessors based on that object value.
*/
public class ObjectToNumberColumnValueSelector implements ColumnValueSelector<Number>
{
private final ColumnValueSelector<?> selector;
private final ExpressionType desiredType;
@Nullable
private final RowIdSupplier rowIdSupplier;
@Nullable
private Number currentValue;
private long currentRowId = RowIdSupplier.INIT;
/**
* Package-private; create using {@link TypeCastSelectors#makeColumnValueSelector} or
* {@link TypeCastSelectors#wrapColumnValueSelectorIfNeeded}.
*/
ObjectToNumberColumnValueSelector(
final ColumnValueSelector<?> selector,
final ExpressionType desiredType,
@Nullable final RowIdSupplier rowIdSupplier
)
{
this.selector = selector;
this.desiredType = desiredType;
this.rowIdSupplier = rowIdSupplier;
if (!desiredType.isNumeric()) {
throw DruidException.defensive("Expected numeric type, got[%s]", desiredType);
}
}
@Override
public double getDouble()
{
final Number n = computeIfNeeded();
return n == null ? NullHandling.ZERO_DOUBLE : n.doubleValue();
}
@Override
public float getFloat()
{
final Number n = computeIfNeeded();
return n == null ? NullHandling.ZERO_FLOAT : n.floatValue();
}
@Override
public long getLong()
{
final Number n = computeIfNeeded();
return n == null ? NullHandling.ZERO_LONG : n.longValue();
}
@Override
public boolean isNull()
{
return computeIfNeeded() == null;
}
@Nullable
@Override
public Number getObject()
{
return computeIfNeeded();
}
@Override
public Class<Number> classOfObject()
{
return Number.class;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
inspector.visit("rowIdSupplier", rowIdSupplier);
}
@Nullable
private Number computeIfNeeded()
{
if (rowIdSupplier == null) {
return eval();
} else {
final long rowId = rowIdSupplier.getRowId();
if (currentRowId != rowId) {
currentValue = eval();
currentRowId = rowId;
}
return currentValue;
}
}
@Nullable
private Number eval()
{
return (Number) TypeCastSelectors.bestEffortCoerce(selector.getObject(), desiredType);
}
}

View File

@ -0,0 +1,121 @@
/*
* 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.frame.write.cast;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.RowIdSupplier;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType;
import javax.annotation.Nullable;
public class TypeCastSelectors
{
/**
* Create a {@link ColumnValueSelector} that does its own typecasting if necessary. If typecasting is not necessary,
* returns a selector directly from the underlying {@link ColumnSelectorFactory}.
*
* @param columnSelectorFactory underlying factory
* @param column column name
* @param desiredType desired type of selector. Can be anything except {@link ColumnType#STRING}.
* For strings, use {@link DimensionSelector} rather than {@link ColumnValueSelector}.
*/
public static ColumnValueSelector<?> makeColumnValueSelector(
final ColumnSelectorFactory columnSelectorFactory,
final String column,
final ColumnType desiredType
)
{
final ColumnValueSelector<?> selector = columnSelectorFactory.makeColumnValueSelector(column);
final ColumnCapabilities selectorCapabilities = columnSelectorFactory.getColumnCapabilities(column);
return wrapColumnValueSelectorIfNeeded(
selector,
selectorCapabilities,
columnSelectorFactory.getRowIdSupplier(),
desiredType
);
}
/**
* Wraps a {@link ColumnValueSelector} with a type casting selector if necessary. If typecasting is not necessary,
* returns the original selector.
*
* @param selector selector
* @param selectorCapabilities capabilities for the selector, from {@link ColumnSelectorFactory#getColumnCapabilities}
* @param rowIdSupplier row id supplier, from {@link ColumnSelectorFactory#getRowIdSupplier()}
* @param desiredType desired type for the returned selector
*/
public static ColumnValueSelector<?> wrapColumnValueSelectorIfNeeded(
final ColumnValueSelector<?> selector,
@Nullable final ColumnCapabilities selectorCapabilities,
@Nullable final RowIdSupplier rowIdSupplier,
final ColumnType desiredType
)
{
final ExpressionType desiredExpressionType = ExpressionType.fromColumnType(desiredType);
if (desiredType.is(ValueType.STRING)) {
throw DruidException.defensive("Type[%s] should be read using a DimensionSelector", desiredType);
} else if (desiredType.isNumeric()
&& (selectorCapabilities == null || !selectorCapabilities.isNumeric())) {
// When capabilities are unknown, or known to be non-numeric, fall back to getObject() and explicit typecasting.
// This avoids using primitive numeric accessors (getLong / getDouble / getFloat / isNull) on a selector that
// may not support them.
return new ObjectToNumberColumnValueSelector(selector, desiredExpressionType, rowIdSupplier);
} else if (desiredType.isArray()) {
// Always wrap if desiredType is an array. Even if the underlying selector claims to offer the same type as
// desiredType, it may fail to respect the BaseObjectColumnValueSelector contract. For example, it may return
// List rather than Object[]. (RowBasedColumnSelectorFactory can do this if used incorrectly, i.e., if the
// ColumnInspector declares type ARRAY<X> for a column, but the RowAdapter does not provide Object[].)
return new ObjectToArrayColumnValueSelector(selector, desiredExpressionType, rowIdSupplier);
} else {
// OK to return the original selector.
return selector;
}
}
/**
* Coerce an object to an object compatible with what {@link BaseObjectColumnValueSelector#getObject()} for a column
* of the provided desiredType.
*
* @param obj object
* @param desiredType desired type
*/
@Nullable
public static Object bestEffortCoerce(
@Nullable final Object obj,
@Nullable final ExpressionType desiredType
)
{
if (obj == null || desiredType == null) {
return obj;
}
return ExprEval.ofType(desiredType, obj).value();
}
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.frame.write.columnar;
import org.apache.druid.frame.allocation.MemoryAllocator;
import org.apache.druid.frame.write.UnsupportedColumnTypeException;
import org.apache.druid.frame.write.cast.TypeCastSelectors;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.segment.ColumnSelectorFactory;
@ -102,7 +103,8 @@ public class FrameColumnWriters
)
{
final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(columnName);
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.LONG);
return new LongFrameColumnWriter(selector, allocator, hasNullsForNumericWriter(capabilities));
}
@ -113,7 +115,8 @@ public class FrameColumnWriters
)
{
final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(columnName);
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.FLOAT);
return new FloatFrameColumnWriter(selector, allocator, hasNullsForNumericWriter(capabilities));
}
@ -124,7 +127,8 @@ public class FrameColumnWriters
)
{
final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(columnName);
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.DOUBLE);
return new DoubleFrameColumnWriter(selector, allocator, hasNullsForNumericWriter(capabilities));
}
@ -149,7 +153,8 @@ public class FrameColumnWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.STRING_ARRAY);
return new StringArrayFrameColumnWriterImpl(selector, allocator);
}
@ -159,7 +164,8 @@ public class FrameColumnWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.LONG_ARRAY);
return new LongArrayFrameColumnWriter(selector, allocator);
}
@ -169,7 +175,8 @@ public class FrameColumnWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.FLOAT_ARRAY);
return new FloatArrayFrameColumnWriter(selector, allocator);
}
@ -179,7 +186,8 @@ public class FrameColumnWriters
final String columnName
)
{
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.DOUBLE_ARRAY);
return new DoubleArrayFrameColumnWriter(selector, allocator);
}
@ -199,7 +207,8 @@ public class FrameColumnWriters
throw new ISE("No serde for complexTypeName[%s], cannot write column [%s]", columnTypeName, columnName);
}
final ColumnValueSelector<?> selector = selectorFactory.makeColumnValueSelector(columnName);
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(selectorFactory, columnName, ColumnType.ofComplex(columnTypeName));
return new ComplexFrameColumnWriter(selector, allocator, serde);
}

View File

@ -24,11 +24,8 @@ import org.apache.druid.error.DruidException;
import org.apache.druid.frame.allocation.AppendableMemory;
import org.apache.druid.frame.allocation.MemoryAllocator;
import org.apache.druid.frame.allocation.MemoryRange;
import org.apache.druid.frame.write.FrameWriterUtils;
import org.apache.druid.segment.ColumnValueSelector;
import java.util.List;
/**
* Parent class for the family of writers writing numeric arrays in columnar frames. Since the numeric primitives are
* fixed width, we don't need to store the width of each element. The memory layout of a column written by this writer
@ -119,8 +116,8 @@ public abstract class NumericArrayFrameColumnWriter implements FrameColumnWriter
@Override
public boolean addSelection()
{
List<? extends Number> numericArray = FrameWriterUtils.getNumericArrayFromObject(selector.getObject());
int rowLength = numericArray == null ? 0 : numericArray.size();
final Object[] row = (Object[]) selector.getObject();
int rowLength = row == null ? 0 : row.length;
// Begin memory allocations before writing
if ((long) lastCumulativeRowLength + rowLength > Integer.MAX_VALUE) {
@ -142,7 +139,7 @@ public abstract class NumericArrayFrameColumnWriter implements FrameColumnWriter
final MemoryRange<WritableMemory> rowLengthsCursor = cumulativeRowLengths.cursor();
if (numericArray == null) {
if (row == null) {
rowLengthsCursor.memory().putInt(rowLengthsCursor.start(), -(lastCumulativeRowLength + rowLength) - 1);
} else {
rowLengthsCursor.memory().putInt(rowLengthsCursor.start(), lastCumulativeRowLength + rowLength);
@ -155,7 +152,7 @@ public abstract class NumericArrayFrameColumnWriter implements FrameColumnWriter
final MemoryRange<WritableMemory> rowDataCursor = rowLength > 0 ? rowData.cursor() : null;
for (int i = 0; i < rowLength; ++i) {
final Number element = numericArray.get(i);
final Number element = (Number) row[i];
final long memoryOffset = rowDataCursor.start() + ((long) elementSizeBytes() * i);
if (element == null) {
rowNullityDataCursor.memory()

View File

@ -406,7 +406,7 @@ public abstract class ExprEval<T>
public static ExprEval bestEffortOf(@Nullable Object val)
{
if (val == null) {
return new StringExprEval(null);
return StringExprEval.OF_NULL;
}
if (val instanceof ExprEval) {
return (ExprEval) val;
@ -519,6 +519,12 @@ public abstract class ExprEval<T>
return ofComplex(ExpressionType.UNKNOWN_COMPLEX, val);
}
/**
* Create an eval of the provided type. Coerces the provided object to the desired type.
*
* @param type type, or null to be equivalent to {@link #bestEffortOf(Object)}
* @param value object to be coerced to the type
*/
public static ExprEval ofType(@Nullable ExpressionType type, @Nullable Object value)
{
if (type == null) {
@ -1145,30 +1151,31 @@ public abstract class ExprEval<T>
switch (castTo.getType()) {
case DOUBLE:
return ExprEval.ofDouble(computeNumber());
case LONG:
return ExprEval.ofLong(computeNumber());
case STRING:
return this;
case ARRAY:
if (value == null) {
return new ArrayExprEval(castTo, null);
}
final Number number = computeNumber();
switch (castTo.getElementType().getType()) {
case DOUBLE:
return ExprEval.ofDoubleArray(
new Object[]{number == null ? null : number.doubleValue()}
);
case LONG:
return ExprEval.ofLongArray(
new Object[]{number == null ? null : number.longValue()}
);
case STRING:
return ExprEval.ofStringArray(new Object[]{value});
default:
ExpressionType elementType = (ExpressionType) castTo.getElementType();
return new ArrayExprEval(castTo, new Object[]{castTo(elementType).value()});
ExprType type = castTo.getElementType().getType();
if (type == ExprType.DOUBLE) {
final Number number = computeNumber();
return ExprEval.ofDoubleArray(new Object[]{number == null ? null : number.doubleValue()});
} else if (type == ExprType.LONG) {
final Number number = computeNumber();
return ExprEval.ofLongArray(new Object[]{number == null ? null : number.longValue()});
} else if (type == ExprType.STRING) {
return ExprEval.ofStringArray(new Object[]{value});
}
ExpressionType elementType = (ExpressionType) castTo.getElementType();
return new ArrayExprEval(castTo, new Object[]{castTo(elementType).value()});
case COMPLEX:
if (ExpressionType.NESTED_DATA.equals(castTo)) {
return new NestedDataExprEval(value);

View File

@ -50,7 +50,7 @@ public class StringArrayFieldWriterTest extends InitializedNullHandlingTest
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@Mock
public BaseObjectColumnValueSelector<List<String>> selector;
public BaseObjectColumnValueSelector<Object[]> selector;
private WritableMemory memory;
private FieldWriter fieldWriter;
@ -115,7 +115,8 @@ public class StringArrayFieldWriterTest extends InitializedNullHandlingTest
private void mockSelector(@Nullable final List<String> values)
{
Mockito.when(selector.getObject()).thenReturn(values);
final Object[] arr = values == null ? null : values.toArray();
Mockito.when(selector.getObject()).thenReturn(arr);
}
private long writeToMemory(final FieldWriter writer)

View File

@ -60,7 +60,7 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@Mock
public BaseObjectColumnValueSelector<List<String>> writeSelector;
public BaseObjectColumnValueSelector<Object[]> writeSelector;
private WritableMemory memory;
private FieldWriter fieldWriter;
@ -277,7 +277,8 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest
private void writeToMemory(@Nullable final List<String> values)
{
Mockito.when(writeSelector.getObject()).thenReturn(values);
final Object[] arr = values == null ? null : values.toArray();
Mockito.when(writeSelector.getObject()).thenReturn(arr);
if (fieldWriter.writeTo(memory, MEMORY_POSITION, memory.getCapacity() - MEMORY_POSITION) < 0) {
throw new ISE("Could not write");

View File

@ -0,0 +1,394 @@
/*
* 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.frame.write.cast;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.error.DruidException;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Map;
public class TypeCastSelectorsTest extends InitializedNullHandlingTest
{
private final ColumnSelectorFactory testColumnSelectorFactory = new TestColumnSelectorFactory(
RowSignature.builder()
.add("x", ColumnType.STRING)
.add("y", ColumnType.STRING)
.add("z", ColumnType.STRING)
.add("da", ColumnType.DOUBLE_ARRAY)
.build(),
ImmutableMap.<String, Object>builder()
.put("x", "12.3")
.put("y", "abc")
.put("da", new Object[]{1.2d, 2.3d})
.build() // z is null
);
@Test
public void test_readXAsLong()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "x", ColumnType.LONG);
Assert.assertEquals(12L, selector.getLong());
Assert.assertEquals(12d, selector.getDouble(), 0);
Assert.assertEquals(12f, selector.getFloat(), 0);
Assert.assertFalse(selector.isNull());
Assert.assertEquals(12L, selector.getObject());
}
@Test
public void test_readXAsDouble()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "x", ColumnType.DOUBLE);
Assert.assertEquals(12L, selector.getLong());
Assert.assertEquals(12.3d, selector.getDouble(), 0);
Assert.assertEquals(12.3f, selector.getFloat(), 0);
Assert.assertFalse(selector.isNull());
Assert.assertEquals(12.3d, selector.getObject());
}
@Test
public void test_readXAsFloat()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "x", ColumnType.FLOAT);
Assert.assertEquals(12L, selector.getLong());
Assert.assertEquals(12.3d, selector.getDouble(), 0.001);
Assert.assertEquals(12.3f, selector.getFloat(), 0);
Assert.assertFalse(selector.isNull());
Assert.assertEquals(12.3d, selector.getObject());
}
@Test
public void test_readXAsLongArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "x", ColumnType.LONG_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{12L}, (Object[]) selector.getObject());
}
@Test
public void test_readXAsStringArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "x", ColumnType.STRING_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{"12.3"}, (Object[]) selector.getObject());
}
@Test
public void test_readYAsLong()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "y", ColumnType.LONG);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readYAsDouble()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "y", ColumnType.DOUBLE);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readYAsFloat()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "y", ColumnType.FLOAT);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readYAsLongArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "y", ColumnType.LONG_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{null}, (Object[]) selector.getObject());
}
@Test
public void test_readYAsStringArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "y", ColumnType.STRING_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{"abc"}, (Object[]) selector.getObject());
}
@Test
public void test_readZAsLong()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "z", ColumnType.LONG);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readZAsDouble()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "z", ColumnType.DOUBLE);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readZAsFloat()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "z", ColumnType.FLOAT);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readZAsLongArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "z", ColumnType.LONG_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertNull(selector.getObject());
}
@Test
public void test_readZAsStringArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "z", ColumnType.STRING_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertNull(selector.getObject());
}
@Test
public void test_readDaAsLong()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "da", ColumnType.LONG);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readDaAsDouble()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "da", ColumnType.DOUBLE);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readDaAsFloat()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "da", ColumnType.FLOAT);
Assert.assertEquals(0L, selector.getLong());
Assert.assertEquals(0d, selector.getDouble(), 0);
Assert.assertEquals(0f, selector.getFloat(), 0);
Assert.assertTrue(selector.isNull());
Assert.assertNull(selector.getObject());
}
@Test
public void test_readDaAsLongArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "da", ColumnType.LONG_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{1L, 2L}, (Object[]) selector.getObject());
}
@Test
public void test_readDaAsStringArray()
{
final ColumnValueSelector<?> selector =
TypeCastSelectors.makeColumnValueSelector(testColumnSelectorFactory, "da", ColumnType.STRING_ARRAY);
Assert.assertThrows(DruidException.class, selector::getLong);
Assert.assertThrows(DruidException.class, selector::getDouble);
Assert.assertThrows(DruidException.class, selector::getFloat);
Assert.assertThrows(DruidException.class, selector::isNull);
Assert.assertArrayEquals(new Object[]{"1.2", "2.3"}, (Object[]) selector.getObject());
}
/**
* Implementation that returns a fixed value per column from {@link ColumnValueSelector#getObject()}. Other
* methods, such as {@link ColumnValueSelector#getLong()} throw exceptions. This is meant to help validate
* that those other methods are *not* called.
*/
private static class TestColumnSelectorFactory implements ColumnSelectorFactory
{
private final RowSignature signature;
private final Map<String, Object> columnValues;
public TestColumnSelectorFactory(final RowSignature signature, final Map<String, Object> columnValues)
{
this.signature = signature;
this.columnValues = columnValues;
}
@Override
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
{
throw new UnsupportedOperationException();
}
@Override
public ColumnValueSelector<?> makeColumnValueSelector(String columnName)
{
return new ColumnValueSelector<Object>()
{
@Override
public double getDouble()
{
throw new UnsupportedOperationException("Should not be called");
}
@Override
public float getFloat()
{
throw new UnsupportedOperationException("Should not be called");
}
@Override
public long getLong()
{
throw new UnsupportedOperationException("Should not be called");
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
throw new UnsupportedOperationException("Should not be called");
}
@Override
public boolean isNull()
{
throw new UnsupportedOperationException("Should not be called");
}
@Nullable
@Override
public Object getObject()
{
return columnValues.get(columnName);
}
@Override
public Class<?> classOfObject()
{
throw new UnsupportedOperationException("Should not be called");
}
};
}
@Nullable
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return signature.getColumnCapabilities(column);
}
}
}