tighten up array handling, fix bug with array_slice output type inference (#12914)

This commit is contained in:
Clint Wylie 2022-08-25 00:48:49 -07:00 committed by GitHub
parent 599bdde02a
commit 82ad927087
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 634 additions and 312 deletions

View File

@ -407,7 +407,7 @@ public interface ApplyFunction
return ExprEval.of(null);
}
if (hadEmpty) {
return ExprEval.ofStringArray(new String[0]);
return ExprEval.ofStringArray(new Object[0]);
}
Expr accExpr = argsExpr.get(argsExpr.size() - 1);
@ -576,9 +576,12 @@ public interface ApplyFunction
@Override
public ExprEval match(Object[] values, LambdaExpr expr, SettableLambdaBinding bindings)
{
boolean anyMatch = Arrays.stream(values)
.anyMatch(o -> expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean());
return ExprEval.ofLongBoolean(anyMatch);
for (Object o : values) {
if (expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean()) {
return ExprEval.ofLongBoolean(true);
}
}
return ExprEval.ofLongBoolean(false);
}
}
@ -599,9 +602,12 @@ public interface ApplyFunction
@Override
public ExprEval match(Object[] values, LambdaExpr expr, SettableLambdaBinding bindings)
{
boolean allMatch = Arrays.stream(values)
.allMatch(o -> expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean());
return ExprEval.ofLongBoolean(allMatch);
for (Object o : values) {
if (!expr.eval(bindings.withBinding(expr.getIdentifier(), o)).asBoolean()) {
return ExprEval.ofLongBoolean(false);
}
}
return ExprEval.ofLongBoolean(true);
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.math.expr;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -30,8 +29,6 @@ import java.util.List;
*/
public class Evals
{
private static final Logger log = new Logger(Evals.class);
public static boolean isAllConstants(Expr... exprs)
{
return isAllConstants(Arrays.asList(exprs));
@ -71,4 +68,16 @@ public class Evals
{
return !NullHandling.isNullOrEquivalent(x) && Boolean.parseBoolean(x);
}
/**
* Call {@link Object#toString()} on a non-null value
*/
@Nullable
public static String asString(@Nullable Object o)
{
if (o == null) {
return null;
}
return o.toString();
}
}

View File

@ -161,22 +161,28 @@ public abstract class ExprEval<T>
}
if (coercedType == Long.class || coercedType == Integer.class) {
return new NonnullPair<>(
ExpressionType.LONG_ARRAY,
val.stream().map(x -> x != null ? ExprEval.ofType(ExpressionType.LONG, x).value() : null).toArray()
);
Object[] array = new Object[val.size()];
int i = 0;
for (Object o : val) {
array[i++] = o == null ? null : ExprEval.ofType(ExpressionType.LONG, o).value();
}
return new NonnullPair<>(ExpressionType.LONG_ARRAY, array);
}
if (coercedType == Float.class || coercedType == Double.class) {
return new NonnullPair<>(
ExpressionType.DOUBLE_ARRAY,
val.stream().map(x -> x != null ? ExprEval.ofType(ExpressionType.DOUBLE, x).value() : null).toArray()
);
Object[] array = new Object[val.size()];
int i = 0;
for (Object o : val) {
array[i++] = o == null ? null : ExprEval.ofType(ExpressionType.DOUBLE, o).value();
}
return new NonnullPair<>(ExpressionType.DOUBLE_ARRAY, array);
}
// default to string
return new NonnullPair<>(
ExpressionType.STRING_ARRAY,
val.stream().map(x -> x != null ? ExprEval.ofType(ExpressionType.STRING, x).value() : null).toArray()
);
Object[] array = new Object[val.size()];
int i = 0;
for (Object o : val) {
array[i++] = o == null ? null : ExprEval.ofType(ExpressionType.STRING, o).value();
}
return new NonnullPair<>(ExpressionType.STRING_ARRAY, array);
}
if (homogenizeMultiValueStrings) {
return new NonnullPair<>(ExpressionType.STRING_ARRAY, new Object[]{null});
@ -297,7 +303,7 @@ public abstract class ExprEval<T>
return new DoubleExprEval(doubleValue);
}
public static ExprEval ofLongArray(@Nullable Long[] longValue)
public static ExprEval ofLongArray(@Nullable Object[] longValue)
{
if (longValue == null) {
return ArrayExprEval.OF_NULL_LONG;
@ -305,7 +311,7 @@ public abstract class ExprEval<T>
return new ArrayExprEval(ExpressionType.LONG_ARRAY, longValue);
}
public static ExprEval ofDoubleArray(@Nullable Double[] doubleValue)
public static ExprEval ofDoubleArray(@Nullable Object[] doubleValue)
{
if (doubleValue == null) {
return ArrayExprEval.OF_NULL_DOUBLE;
@ -313,7 +319,7 @@ public abstract class ExprEval<T>
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, doubleValue);
}
public static ExprEval ofStringArray(@Nullable String[] stringValue)
public static ExprEval ofStringArray(@Nullable Object[] stringValue)
{
if (stringValue == null) {
return ArrayExprEval.OF_NULL_STRING;
@ -388,16 +394,60 @@ public abstract class ExprEval<T>
return new StringExprEval(String.valueOf(val));
}
if (val instanceof Long[]) {
return new ArrayExprEval(ExpressionType.LONG_ARRAY, (Long[]) val);
final Long[] inputArray = (Long[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.LONG_ARRAY, array);
}
if (val instanceof long[]) {
final long[] longArray = (long[]) val;
final Object[] array = new Object[longArray.length];
for (int i = 0; i < longArray.length; i++) {
array[i] = longArray[i];
}
return new ArrayExprEval(ExpressionType.LONG_ARRAY, array);
}
if (val instanceof Double[]) {
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, (Double[]) val);
final Double[] inputArray = (Double[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, array);
}
if (val instanceof double[]) {
final double[] inputArray = (double[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, array);
}
if (val instanceof Float[]) {
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, Arrays.stream((Float[]) val).map(Float::doubleValue).toArray());
final Float[] inputArray = (Float[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i] != null ? inputArray[i].doubleValue() : null;
}
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, array);
}
if (val instanceof float[]) {
final float[] inputArray = (float[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, array);
}
if (val instanceof String[]) {
return new ArrayExprEval(ExpressionType.STRING_ARRAY, (String[]) val);
final String[] inputArray = (String[]) val;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.STRING_ARRAY, array);
}
if (val instanceof List || val instanceof Object[]) {
@ -429,7 +479,12 @@ public abstract class ExprEval<T>
case STRING:
// not all who claim to be "STRING" are always a String, prepare ourselves...
if (value instanceof String[]) {
return new ArrayExprEval(ExpressionType.STRING_ARRAY, (String[]) value);
final String[] inputArray = (String[]) value;
final Object[] array = new Object[inputArray.length];
for (int i = 0; i < inputArray.length; i++) {
array[i] = inputArray[i];
}
return new ArrayExprEval(ExpressionType.STRING_ARRAY, array);
}
if (value instanceof Object[]) {
return bestEffortOf(value);
@ -437,10 +492,7 @@ public abstract class ExprEval<T>
if (value instanceof List) {
return bestEffortOf(value);
}
if (value == null) {
return of(null);
}
return of(String.valueOf(value));
return of(Evals.asString(value));
case LONG:
if (value instanceof Number) {
return ofLong((Number) value);
@ -571,12 +623,7 @@ public abstract class ExprEval<T>
public String asString()
{
if (!stringValueCached) {
if (value == null) {
stringValue = null;
} else {
stringValue = String.valueOf(value);
}
stringValue = Evals.asString(value);
stringValueCached = true;
}
@ -630,15 +677,6 @@ public abstract class ExprEval<T>
@Nullable
public abstract Object[] asArray();
@Nullable
public abstract String[] asStringArray();
@Nullable
public abstract Long[] asLongArray();
@Nullable
public abstract Double[] asDoubleArray();
public abstract ExprEval castTo(ExpressionType castTo);
public abstract Expr toExpr();
@ -668,27 +706,6 @@ public abstract class ExprEval<T>
return value.doubleValue();
}
@Nullable
@Override
public String[] asStringArray()
{
return isNumericNull() ? null : new String[] {value.toString()};
}
@Nullable
@Override
public Long[] asLongArray()
{
return isNumericNull() ? null : new Long[] {value.longValue()};
}
@Nullable
@Override
public Double[] asDoubleArray()
{
return isNumericNull() ? null : new Double[] {value.doubleValue()};
}
@Override
public boolean isNumericNull()
{
@ -721,7 +738,7 @@ public abstract class ExprEval<T>
@Override
public Object[] asArray()
{
return asDoubleArray();
return isNumericNull() ? null : new Object[] {value.doubleValue()};
}
@Override
@ -741,11 +758,11 @@ public abstract class ExprEval<T>
case ARRAY:
switch (castTo.getElementType().getType()) {
case DOUBLE:
return ExprEval.ofDoubleArray(asDoubleArray());
return ExprEval.ofDoubleArray(asArray());
case LONG:
return ExprEval.ofLongArray(asLongArray());
return ExprEval.ofLongArray(value == null ? null : new Object[] {value.longValue()});
case STRING:
return ExprEval.ofStringArray(asStringArray());
return ExprEval.ofStringArray(value == null ? null : new Object[] {value.toString()});
}
}
throw new IAE("invalid type " + castTo);
@ -786,14 +803,7 @@ public abstract class ExprEval<T>
@Override
public Object[] asArray()
{
return asLongArray();
}
@Nullable
@Override
public Long[] asLongArray()
{
return isNumericNull() ? null : new Long[]{value.longValue()};
return isNumericNull() ? null : new Object[] {value.longValue()};
}
@Override
@ -813,11 +823,11 @@ public abstract class ExprEval<T>
case ARRAY:
switch (castTo.getElementType().getType()) {
case DOUBLE:
return ExprEval.ofDoubleArray(asDoubleArray());
return ExprEval.ofDoubleArray(value == null ? null : new Object[] {value.doubleValue()});
case LONG:
return ExprEval.ofLongArray(asLongArray());
return ExprEval.ofLongArray(asArray());
case STRING:
return ExprEval.ofStringArray(asStringArray());
return ExprEval.ofStringArray(value == null ? null : new Object[] {value.toString()});
}
}
throw new IAE("invalid type " + castTo);
@ -905,7 +915,7 @@ public abstract class ExprEval<T>
@Override
public Object[] asArray()
{
return asStringArray();
return value == null ? null : new Object[] {value};
}
private int computeInt()
@ -969,27 +979,6 @@ public abstract class ExprEval<T>
return booleanValue;
}
@Nullable
@Override
public String[] asStringArray()
{
return value == null ? null : new String[] {value};
}
@Nullable
@Override
public Long[] asLongArray()
{
return value == null ? null : new Long[] {computeLong()};
}
@Nullable
@Override
public Double[] asDoubleArray()
{
return value == null ? null : new Double[] {computeDouble()};
}
@Override
public final ExprEval castTo(ExpressionType castTo)
{
@ -1003,11 +992,11 @@ public abstract class ExprEval<T>
case ARRAY:
switch (castTo.getElementType().getType()) {
case DOUBLE:
return ExprEval.ofDoubleArray(asDoubleArray());
return ExprEval.ofDoubleArray(value == null ? null : new Object[] {computeDouble()});
case LONG:
return ExprEval.ofLongArray(asLongArray());
return ExprEval.ofLongArray(value == null ? null : new Object[] {computeLong()});
case STRING:
return ExprEval.ofStringArray(asStringArray());
return ExprEval.ofStringArray(value == null ? null : new Object[] {value});
}
}
throw new IAE("invalid type " + castTo);
@ -1050,11 +1039,7 @@ public abstract class ExprEval<T>
if (value == null) {
cacheStringValue(null);
} else if (value.length == 1) {
if (value[0] == null) {
cacheStringValue(null);
} else {
cacheStringValue(String.valueOf(value[0]));
}
cacheStringValue(Evals.asString(value[0]));
} else {
cacheStringValue(Arrays.toString(value));
}
@ -1166,68 +1151,6 @@ public abstract class ExprEval<T>
return value;
}
@Nullable
@Override
public String[] asStringArray()
{
if (value != null) {
if (arrayType.getElementType().is(ExprType.STRING)) {
return Arrays.stream(value).map(v -> (String) v).toArray(String[]::new);
} else if (arrayType.getElementType().isNumeric()) {
return Arrays.stream(value).map(x -> x != null ? x.toString() : null).toArray(String[]::new);
}
}
return null;
}
@Nullable
@Override
public Long[] asLongArray()
{
if (arrayType.getElementType().is(ExprType.LONG)) {
return Arrays.stream(value).map(v -> (Long) v).toArray(Long[]::new);
} else if (arrayType.getElementType().is(ExprType.DOUBLE)) {
return value == null ? null : Arrays.stream(value).map(v -> ((Double) v).longValue()).toArray(Long[]::new);
} else if (arrayType.getElementType().is(ExprType.STRING)) {
return Arrays.stream(value).map(v -> {
if (v == null) {
return null;
}
Long lv = GuavaUtils.tryParseLong((String) v);
if (lv == null) {
Double d = Doubles.tryParse((String) v);
if (d != null) {
lv = d.longValue();
}
}
return lv;
}).toArray(Long[]::new);
}
return null;
}
@Nullable
@Override
public Double[] asDoubleArray()
{
if (arrayType.getElementType().is(ExprType.DOUBLE)) {
return Arrays.stream(value).map(v -> (Double) v).toArray(Double[]::new);
} else if (arrayType.getElementType().is(ExprType.LONG)) {
return value == null ? null : Arrays.stream(value).map(v -> ((Long) v).doubleValue()).toArray(Double[]::new);
} else if (arrayType.getElementType().is(ExprType.STRING)) {
if (value == null) {
return null;
}
return Arrays.stream(value).map(val -> {
if (val == null) {
return null;
}
return Doubles.tryParse((String) val);
}).toArray(Double[]::new);
}
return new Double[0];
}
@Override
public ExprEval castTo(ExpressionType castTo)
{
@ -1346,27 +1269,6 @@ public abstract class ExprEval<T>
return new Object[0];
}
@Nullable
@Override
public String[] asStringArray()
{
return new String[0];
}
@Nullable
@Override
public Long[] asLongArray()
{
return new Long[0];
}
@Nullable
@Override
public Double[] asDoubleArray()
{
return new Double[0];
}
@Override
public ExprEval castTo(ExpressionType castTo)
{

View File

@ -54,7 +54,6 @@ import java.util.function.BinaryOperator;
import java.util.function.DoubleBinaryOperator;
import java.util.function.LongBinaryOperator;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Base interface describing the mechanism used to evaluate a {@link FunctionExpr}. All {@link Function} implementations
@ -507,12 +506,13 @@ public interface Function
if (!scalarExpr.type().equals(arrayExpr.elementType())) {
// try to cast
ExprEval coerced = scalarExpr.castTo(arrayExpr.elementType());
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), coerced.value()).toArray());
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), coerced.value()));
}
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), scalarExpr.value()).toArray());
return ExprEval.ofArray(arrayType, add(arrayType.getElementType(), arrayExpr.asArray(), scalarExpr.value()));
}
abstract <T> Stream<T> add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val);
abstract <T> Object[] add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val);
}
/**
@ -558,13 +558,13 @@ public interface Function
if (!lhsExpr.asArrayType().equals(rhsExpr.asArrayType())) {
// try to cast if they types don't match
ExprEval coerced = rhsExpr.castTo(arrayType);
ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), coerced.asArray()).toArray());
ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), coerced.asArray()));
}
return ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), rhsExpr.asArray()).toArray());
return ExprEval.ofArray(arrayType, merge(arrayType.getElementType(), lhsExpr.asArray(), rhsExpr.asArray()));
}
abstract <T> Stream<T> merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2);
abstract <T> Object[] merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2);
}
abstract class ReduceFunction implements Function
@ -3406,11 +3406,14 @@ public interface Function
}
@Override
<T> Stream<T> add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
<T> Object[] add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
{
List<T> l = new ArrayList<>(Arrays.asList(array));
l.add(val);
return l.stream();
final Object[] output = new Object[array.length + 1];
for (int i = 0; i < array.length; i++) {
output[i] = array[i];
}
output[array.length] = val;
return output;
}
}
@ -3435,11 +3438,14 @@ public interface Function
}
@Override
<T> Stream<T> add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
<T> Object[] add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
{
List<T> l = new ArrayList<>(Arrays.asList(array));
l.add(0, val);
return l.stream();
final Object[] output = new Object[array.length + 1];
output[0] = val;
for (int i = 0; i < array.length; i++) {
output[i + 1] = array[i];
}
return output;
}
}
@ -3452,11 +3458,16 @@ public interface Function
}
@Override
<T> Stream<T> merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2)
<T> Object[] merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2)
{
List<T> l = new ArrayList<>(Arrays.asList(array1));
l.addAll(Arrays.asList(array2));
return l.stream();
final Object[] output = new Object[array1.length + array2.length];
for (int i = 0; i < array1.length; i++) {
output[i] = array1[i];
}
for (int i = array1.length, j = 0; j < array2.length; i++, j++) {
output[i] = array2[j];
}
return output;
}
}
@ -3469,12 +3480,12 @@ public interface Function
}
@Override
<T> Stream<T> add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
<T> Object[] add(TypeSignature<ExprType> elementType, T[] array, @Nullable T val)
{
Set<T> l = new TreeSet<>(elementType.getNullableStrategy());
l.addAll(Arrays.asList(array));
l.add(val);
return l.stream();
Set<T> set = new TreeSet<>(elementType.getNullableStrategy());
set.addAll(Arrays.asList(array));
set.add(val);
return set.toArray();
}
}
@ -3487,12 +3498,12 @@ public interface Function
}
@Override
<T> Stream<T> merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2)
<T> Object[] merge(TypeSignature<ExprType> elementType, T[] array1, T[] array2)
{
Set<T> l = new TreeSet<>(elementType.getNullableStrategy());
l.addAll(Arrays.asList(array1));
l.addAll(Arrays.asList(array2));
return l.stream();
return l.toArray();
}
}
@ -3574,7 +3585,8 @@ public interface Function
@Override
public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List<Expr> args)
{
return args.get(0).getOutputType(inspector);
ExpressionType arrayType = args.get(0).getOutputType(inspector);
return Optional.ofNullable(ExpressionType.asArrayType(arrayType)).orElse(arrayType);
}
@Override

View File

@ -19,11 +19,10 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExpressionType;
import java.util.Arrays;
public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor<String[]>
{
public CastToStringVectorProcessor(ExprVectorProcessor<?> delegate)
@ -35,9 +34,12 @@ public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor
public ExprEvalVector<String[]> evalVector(Expr.VectorInputBinding bindings)
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
return new ExprEvalStringVector(
Arrays.stream(result.getObjectVector()).map(x -> x != null ? x.toString() : null).toArray(String[]::new)
);
final Object[] objects = result.getObjectVector();
final String[] output = new String[objects.length];
for (int i = 0; i < objects.length; i++) {
output[i] = Evals.asString(objects[i]);
}
return new ExprEvalStringVector(output);
}
@Override

View File

@ -26,18 +26,72 @@ import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import javax.annotation.Nullable;
/**
* Native Druid types.
*
* @see TypeSignature
*/
@JsonSerialize(using = ToStringSerializer.class)
public class ColumnType extends BaseTypeSignature<ValueType>
{
/**
* Druid string type. Values will be represented as {@link String} or {@link java.util.List<String>} in the case
* of multi-value string columns. {@link ColumnType} has insufficient information to distinguish between single
* and multi-value strings, this requires a specific {@link TypeSignature} implementation which is supplied by
* segments, 'ColumnCapabilities', which is available at a higher layer and provides the method 'hasMultipleValues'.
*
* @see ValueType#STRING
*/
public static final ColumnType STRING = new ColumnType(ValueType.STRING, null, null);
/**
* Druid 64-bit integer number primitve type. Values will be represented as Java long or {@link Long}.
*
* @see ValueType#LONG
*/
public static final ColumnType LONG = new ColumnType(ValueType.LONG, null, null);
/**
* Druid 64-bit double precision floating point number primitive type. Values will be represented as Java double or
* {@link Double}.
*
* @see ValueType#DOUBLE
*/
public static final ColumnType DOUBLE = new ColumnType(ValueType.DOUBLE, null, null);
/**
* Druid 32-bit single precision floating point number primitive type. Values will be represented as Java float or
* {@link Float}.
*
* @see ValueType#FLOAT
*/
public static final ColumnType FLOAT = new ColumnType(ValueType.FLOAT, null, null);
// currently, arrays only come from expressions or aggregators
// and there are no native float expressions (or aggs which produce float arrays)
/**
* An array of Strings. Values will be represented as Object[]
* @see ValueType#ARRAY
* @see ValueType#STRING
*/
public static final ColumnType STRING_ARRAY = new ColumnType(ValueType.ARRAY, null, STRING);
/**
* An array of Longs. Values will be represented as Object[] or long[]
* @see ValueType#ARRAY
* @see ValueType#LONG
*/
public static final ColumnType LONG_ARRAY = new ColumnType(ValueType.ARRAY, null, LONG);
/**
* An array of Doubles. Values will be represented as Object[] or double[].
* @see ValueType#ARRAY
* @see ValueType#DOUBLE
*/
public static final ColumnType DOUBLE_ARRAY = new ColumnType(ValueType.ARRAY, null, DOUBLE);
/**
* Placeholder type for an "unknown" complex, which is used when the complex type name was "lost" or unavailable for
* whatever reason, to indicate an opaque type that cannot be generically handled with normal complex type handling
* mechanisms. Prefer to use a {@link ColumnType} with the {@link #complexTypeName} set for most complex type matters
* if at all possible.
*
* @see ValueType#COMPLEX
*/
public static final ColumnType UNKNOWN_COMPLEX = new ColumnType(ValueType.COMPLEX, null, null);
@JsonCreator

View File

@ -63,23 +63,38 @@ public enum ValueType implements TypeDescriptor
* String object type. This type may be used as a grouping key, an input to certain types of complex sketch
* aggregators, and as an input to expression virtual columns. String types might potentially be 'multi-valued' when
* stored in segments, and contextually at various layers of query processing, but this information is not available
* through this enum alone, and must be accompany this type indicator to properly handle.
* at this level.
*
* Strings are typically represented as {@link String}. Multi-value strings appear as {@link java.util.List<String>}
* when necessary to represent multiple values, and can vary between string and list from one row to the next.
*/
STRING,
/**
* Placeholder for arbitrary 'complex' types, which have a corresponding serializer/deserializer implementation. Note
* that knowing a type is complex alone isn't enough information to work with it directly, and additional information
* in the form of a type name that is registered in the complex type registry must be available to make this type
* meaningful. This type is not currently supported as a grouping key for aggregations, and may not be used as an
* input to expression virtual columns, and might only be supported by the specific aggregators crafted to handle
* this complex type.
* in the form of a type name which must be registered in the complex type registry. Complex types are not currently
* supported as a grouping key for aggregations. Complex types can be used as inputs to aggregators, in cases where
* the specific aggregator supports the specific complex type. Filtering on these types with standard filters is not
* well supported, and will be treated as null values.
*
* These types are represented by the individual Java type associated with the complex type name as defined in the
* type registry.
*/
COMPLEX,
/**
* Placeholder for arbitrary arrays of other {@link ValueType}. This type is not currently supported as a grouping
* key for aggregations, cannot be used as an input for numerical primitive aggregations such as sums, and may have
* limited support as an input among complex type sketch aggregators.
* Placeholder for arbitrary arrays of other {@link ValueType}. This type has limited support as a grouping
* key for aggregations, ARRAY of STRING, LONG, DOUBLE, and FLOAT are supported, but ARRAY types in general are not.
* ARRAY types cannot be used as an input for numerical primitive aggregations such as sums, and have limited support
* as an input among complex type sketch aggregators.
*
* There are currently no native ARRAY typed columns, but they may be produced by expression virtual columns,
* aggregators, and post-aggregators.
*
* Arrays are represented as Object[], long[], double[], or float[]. The preferred type is Object[], since the
* expression system is the main consumer of arrays, and the expression system uses Object[] internally. Some code
* represents arrays in other ways; in particular the groupBy engine and SQL result layer. Over time we expect these
* usages to migrate to Object[], long[], double[], and float[].
*/
ARRAY;

View File

@ -20,6 +20,7 @@
package org.apache.druid.math.expr;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.StringUtils;
@ -131,6 +132,7 @@ public class ExprEvalTest extends InitializedNullHandlingTest
public void testLongArraySerde()
{
assertExpr(0, new Long[]{1L, 2L, 3L});
assertExpr(0, new long[]{1L, 2L, 3L});
assertExpr(1234, new Long[]{1L, 2L, null, 3L});
assertExpr(1234, new Long[]{});
}
@ -165,6 +167,7 @@ public class ExprEvalTest extends InitializedNullHandlingTest
public void testDoubleArraySerde()
{
assertExpr(0, new Double[]{1.1, 2.2, 3.3});
assertExpr(0, new double[]{1.1, 2.2, 3.3});
assertExpr(1234, new Double[]{1.1, 2.2, null, 3.3});
assertExpr(1234, new Double[]{});
}
@ -334,12 +337,12 @@ public class ExprEvalTest extends InitializedNullHandlingTest
{
ExprEval someStringArray = ExprEval.ofStringArray(new String[]{"1", "2", "foo", null, "3.3"});
Assert.assertArrayEquals(
new Long[]{1L, 2L, null, null, 3L},
someStringArray.asLongArray()
new Object[]{1L, 2L, NullHandling.defaultLongValue(), NullHandling.defaultLongValue(), 3L},
someStringArray.castTo(ExpressionType.LONG_ARRAY).asArray()
);
Assert.assertArrayEquals(
new Double[]{1.0, 2.0, null, null, 3.3},
someStringArray.asDoubleArray()
new Object[]{1.0, 2.0, NullHandling.defaultDoubleValue(), NullHandling.defaultDoubleValue(), 3.3},
someStringArray.castTo(ExpressionType.DOUBLE_ARRAY).asArray()
);
}

View File

@ -41,6 +41,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator;
import org.apache.druid.query.aggregation.post.ExpressionPostAggregator;
import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.filter.NotDimFilter;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.query.groupby.GroupByQuery;
@ -65,6 +66,7 @@ import javax.annotation.Nonnull;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -944,6 +946,74 @@ public class MSQSelectTest extends MSQTestBase
.verifyResults();
}
@Test
public void testGroupByArrayWithMultiValueMvToArray()
{
Map<String, Object> context = ImmutableMap.<String, Object>builder()
.putAll(DEFAULT_MSQ_CONTEXT)
.put("groupByEnableMultiValueUnnesting", true)
.build();
RowSignature rowSignature = RowSignature.builder()
.add("EXPR$0", ColumnType.STRING_ARRAY)
.add("cnt1", ColumnType.LONG)
.build();
ArrayList<Object[]> expected = new ArrayList<>();
expected.add(new Object[]{Collections.singletonList(null), !useDefault ? 2L : 3L});
if (!useDefault) {
expected.add(new Object[]{Collections.singletonList(""), 1L});
}
expected.addAll(ImmutableList.of(
new Object[]{Arrays.asList("a", "b"), 1L},
new Object[]{Arrays.asList("b", "c"), 1L},
new Object[]{Collections.singletonList("d"), 1L}
));
testSelectQuery()
.setSql("select MV_TO_ARRAY(dim3), count(*) as cnt1 from foo group by MV_TO_ARRAY(dim3)")
.setQueryContext(context)
.setExpectedMSQSpec(MSQSpec.builder()
.query(GroupByQuery.builder()
.setDataSource(CalciteTests.DATASOURCE1)
.setInterval(querySegmentSpec(Filtration.eternity()))
.setGranularity(Granularities.ALL)
.setDimensions(
dimensions(
new DefaultDimensionSpec(
"v0",
"d0",
ColumnType.STRING_ARRAY
)
)
)
.setVirtualColumns(
new ExpressionVirtualColumn(
"v0",
"mv_to_array(\"dim3\")",
ColumnType.STRING_ARRAY,
TestExprMacroTable.INSTANCE
)
)
.setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
.setContext(context)
.build()
)
.columnMappings(
new ColumnMappings(
ImmutableList.of(
new ColumnMapping("d0", "EXPR$0"),
new ColumnMapping("a0", "cnt1")
)
)
)
.tuningConfig(MSQTuningConfig.defaultConfig())
.build())
.setExpectedRowSignature(rowSignature)
.setExpectedResultRows(expected)
.verifyResults();
}
@Test
public void testGroupByWithMultiValueMvToArrayWithoutGroupByEnable()
{

View File

@ -147,9 +147,9 @@ public class FrameWriterUtils
for (int i = 0; i < ((List<?>) row).size(); i++) {
retVal.add(getUtf8ByteBufferFromString(((List<String>) row).get(i)));
}
} else if (row instanceof String[]) {
for (String value : (String[]) row) {
retVal.add(getUtf8ByteBufferFromString(value));
} else if (row instanceof Object[]) {
for (Object value : (Object[]) row) {
retVal.add(getUtf8ByteBufferFromString((String) value));
}
} else if (row instanceof ComparableStringArray) {
for (String value : ((ComparableStringArray) row).getDelegate()) {

View File

@ -141,26 +141,26 @@ public class ExpressionFilter implements Filter
if (eval.type().isArray()) {
switch (eval.elementType().getType()) {
case LONG:
final Long[] lResult = eval.asLongArray();
final Object[] lResult = eval.asArray();
if (lResult == null) {
return false;
}
return Arrays.stream(lResult).filter(Objects::nonNull).anyMatch(Evals::asBoolean);
return Arrays.stream(lResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((long) o));
case STRING:
final String[] sResult = eval.asStringArray();
final Object[] sResult = eval.asArray();
if (sResult == null) {
return false;
}
return Arrays.stream(sResult).anyMatch(Evals::asBoolean);
return Arrays.stream(sResult).anyMatch(o -> Evals.asBoolean((String) o));
case DOUBLE:
final Double[] dResult = eval.asDoubleArray();
final Object[] dResult = eval.asArray();
if (dResult == null) {
return false;
}
return Arrays.stream(dResult).filter(Objects::nonNull).anyMatch(Evals::asBoolean);
return Arrays.stream(dResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((double) o));
}
}
return eval.asBoolean();

View File

@ -99,7 +99,13 @@ public class ExpressionTransform implements Transform
@Override
public Object eval(final Row row)
{
return ExpressionSelectors.coerceEvalToSelectorObject(
// this will need adjusted if we want to allow expression transforms to produce true arrays. Currently, calling
// this method will coerce any expression output into:
// - the expression value if the value is not an array
// - the single array element if the value is an array with 1 element
// - a list with all of the array elements if the value is an array with more than 1 element
// and so is tuned towards multi-value strings
return ExpressionSelectors.coerceEvalToObjectOrList(
expr.eval(InputBindings.forFunction(name -> getValueFromRow(row, name)))
);
}

View File

@ -21,6 +21,7 @@ package org.apache.druid.segment.virtual;
import com.google.common.base.Predicate;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.filter.ValueMatcher;
@ -71,26 +72,22 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
String getValue(ExprEval evaluated)
{
assert !evaluated.isArray();
return NullHandling.emptyToNullIfNeeded(evaluated.asString());
return evaluated.asString();
}
List<String> getArray(ExprEval evaluated)
List<String> getArrayAsList(ExprEval evaluated)
{
assert evaluated.isArray();
//noinspection ConstantConditions
return Arrays.stream(evaluated.asStringArray())
.map(NullHandling::emptyToNullIfNeeded)
return Arrays.stream(evaluated.asArray())
.map(Evals::asString)
.collect(Collectors.toList());
}
@Nullable
String getArrayValue(ExprEval evaluated, int i)
{
assert evaluated.isArray();
String[] stringArray = evaluated.asStringArray();
//noinspection ConstantConditions because of assert statement above
assert i < stringArray.length;
return NullHandling.emptyToNullIfNeeded(stringArray[i]);
return getArrayElement(evaluated, i);
}
@Override
@ -121,7 +118,7 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
return getArrayValue(evaluated, id);
}
assert id == 0;
return NullHandling.emptyToNullIfNeeded(evaluated.asString());
return evaluated.asString();
}
@Override
@ -134,7 +131,7 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
{
ExprEval evaluated = getEvaluated();
if (evaluated.isArray()) {
List<String> array = getArray(evaluated);
List<String> array = getArrayAsList(evaluated);
return array.stream().anyMatch(x -> Objects.equals(x, value));
}
return Objects.equals(getValue(evaluated), value);
@ -158,7 +155,7 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
{
ExprEval evaluated = getEvaluated();
if (evaluated.isArray()) {
List<String> array = getArray(evaluated);
List<String> array = getArrayAsList(evaluated);
return array.stream().anyMatch(x -> predicate.apply(x));
}
return predicate.apply(getValue(evaluated));
@ -198,7 +195,7 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
{
ExprEval evaluated = getEvaluated();
if (evaluated.isArray()) {
return getArray(evaluated);
return getArrayAsList(evaluated);
}
return getValue(evaluated);
}
@ -230,21 +227,18 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
}
@Override
List<String> getArray(ExprEval evaluated)
List<String> getArrayAsList(ExprEval evaluated)
{
assert evaluated.isArray();
return Arrays.stream(evaluated.asStringArray())
.map(x -> extractionFn.apply(NullHandling.emptyToNullIfNeeded(x)))
return Arrays.stream(evaluated.asArray())
.map(x -> extractionFn.apply(Evals.asString(x)))
.collect(Collectors.toList());
}
@Override
String getArrayValue(ExprEval evaluated, int i)
{
assert evaluated.isArray();
String[] stringArray = evaluated.asStringArray();
assert i < stringArray.length;
return extractionFn.apply(NullHandling.emptyToNullIfNeeded(stringArray[i]));
return extractionFn.apply(ExpressionMultiValueDimensionSelector.getArrayElement(evaluated, i));
}
@Override
@ -254,4 +248,14 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector
inspector.visit("extractionFn", extractionFn);
}
}
@Nullable
private static String getArrayElement(ExprEval eval, int i)
{
final Object[] stringArray = eval.asArray();
if (stringArray == null) {
return null;
}
return Evals.asString(stringArray[i]);
}
}

View File

@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionProcessing;
@ -109,7 +110,66 @@ public class ExpressionSelectors
{
// No need for null check on getObject() since baseSelector impls will never return null.
ExprEval eval = baseSelector.getObject();
return coerceEvalToSelectorObject(eval);
return eval.value();
}
@Override
public Class classOfObject()
{
return Object.class;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("baseSelector", baseSelector);
}
};
}
public static ColumnValueSelector makeStringColumnValueSelector(
ColumnSelectorFactory columnSelectorFactory,
Expr expression
)
{
final ColumnValueSelector<ExprEval> baseSelector = makeExprEvalSelector(columnSelectorFactory, expression);
return new ColumnValueSelector()
{
@Override
public double getDouble()
{
// No Assert for null handling as baseSelector already have it.
return baseSelector.getDouble();
}
@Override
public float getFloat()
{
// No Assert for null handling as baseSelector already have it.
return baseSelector.getFloat();
}
@Override
public long getLong()
{
// No Assert for null handling as baseSelector already have it.
return baseSelector.getLong();
}
@Override
public boolean isNull()
{
return baseSelector.isNull();
}
@Nullable
@Override
public Object getObject()
{
// No need for null check on getObject() since baseSelector impls will never return null.
ExprEval eval = baseSelector.getObject();
return coerceEvalToObjectOrList(eval);
}
@Override
@ -208,8 +268,14 @@ public class ExpressionSelectors
if (baseSelector instanceof ConstantExprEvalSelector) {
// Optimization for dimension selectors on constants.
if (plan.is(ExpressionPlan.Trait.NON_SCALAR_OUTPUT)) {
final String[] value = baseSelector.getObject().asStringArray();
return DimensionSelector.multiConstant(value == null ? null : Arrays.asList(value), extractionFn);
final Object[] value = baseSelector.getObject().asArray();
final List<String> stringList;
if (value != null) {
stringList = Arrays.stream(value).map(Evals::asString).collect(Collectors.toList());
} else {
stringList = null;
}
return DimensionSelector.multiConstant(stringList, extractionFn);
}
return DimensionSelector.constant(baseSelector.getObject().asString(), extractionFn);
} else if (baseSelector instanceof NilColumnValueSelector) {
@ -460,17 +526,27 @@ public class ExpressionSelectors
}
/**
* Coerces {@link ExprEval} value back to selector friendly {@link List} if the evaluated expression result is an
* array type
* Coerces {@link ExprEval} value back to a {@link ColumnType#STRING} selector friendly value, converting into:
* - the expression value if the value is not an array
* - the single array element if the value is an array with 1 element
* - a list with all of the array elements if the value is an array with more than 1 element
* This method is used by {@link #makeStringColumnValueSelector(ColumnSelectorFactory, Expr)}, which is used
* exclusively for making {@link ColumnValueSelector} when an {@link ExpressionVirtualColumn} has STRING output type,
* and by {@link org.apache.druid.segment.transform.ExpressionTransform} which should be reconsidered if we ever
* want to add support for ingestion transforms producing {@link ValueType#ARRAY} typed outputs.
*/
@Nullable
public static Object coerceEvalToSelectorObject(ExprEval eval)
public static Object coerceEvalToObjectOrList(ExprEval eval)
{
if (eval.type().isArray()) {
final Object[] asArray = eval.asArray();
return asArray == null
? null
: Arrays.stream(asArray).collect(Collectors.toList());
if (asArray == null) {
return null;
}
if (asArray.length == 1) {
return asArray[0];
}
return Arrays.stream(asArray).collect(Collectors.toList());
}
return eval.value();
}

View File

@ -42,6 +42,7 @@ import org.apache.druid.segment.VirtualColumn;
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.column.ValueType;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorObjectSelector;
@ -140,6 +141,13 @@ public class ExpressionVirtualColumn implements VirtualColumn
@Override
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
{
final ColumnCapabilities capabilities = capabilities(factory, name);
// we make a special column value selector for values that are expected to be STRING to conform to behavior of
// other single and multi-value STRING selectors, whose getObject is expected to produce a single STRING value
// or List of STRING values.
if (capabilities.is(ValueType.STRING)) {
return ExpressionSelectors.makeStringColumnValueSelector(factory, parsedExpression.get());
}
return ExpressionSelectors.makeColumnValueSelector(factory, parsedExpression.get());
}

View File

@ -599,17 +599,17 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest
{
Assert.assertEquals(
ImmutableList.of(1L, 2L, 3L),
ExpressionSelectors.coerceEvalToSelectorObject(ExprEval.ofLongArray(new Long[]{1L, 2L, 3L}))
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofLongArray(new Long[]{1L, 2L, 3L}))
);
Assert.assertEquals(
ImmutableList.of(1.0, 2.0, 3.0),
ExpressionSelectors.coerceEvalToSelectorObject(ExprEval.ofDoubleArray(new Double[]{1.0, 2.0, 3.0}))
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofDoubleArray(new Double[]{1.0, 2.0, 3.0}))
);
Assert.assertEquals(
ImmutableList.of("a", "b", "c"),
ExpressionSelectors.coerceEvalToSelectorObject(ExprEval.ofStringArray(new String[]{"a", "b", "c"}))
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofStringArray(new String[]{"a", "b", "c"}))
);
List<String> withNulls = new ArrayList<>();
@ -618,7 +618,18 @@ public class ExpressionSelectorsTest extends InitializedNullHandlingTest
withNulls.add("c");
Assert.assertEquals(
withNulls,
ExpressionSelectors.coerceEvalToSelectorObject(ExprEval.ofStringArray(new String[]{"a", null, "c"}))
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofStringArray(new String[]{"a", null, "c"}))
);
Assert.assertNull(
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofLongArray(null))
);
Assert.assertEquals(
1L,
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofLongArray(new Long[]{1L}))
);
Assert.assertNull(
ExpressionSelectors.coerceEvalToObjectOrList(ExprEval.ofLongArray(new Long[]{null}))
);
}

View File

@ -25,6 +25,7 @@ import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.InputBindings;
@ -104,7 +105,7 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
// Evaluate the expression to get out the array elements.
// We can safely pass a noop ObjectBinding if the expression is literal.
ExprEval<?> exprEval = expr.eval(InputBindings.nilBindings());
String[] arrayElements = exprEval.asStringArray();
Object[] arrayElements = exprEval.asArray();
if (arrayElements == null || arrayElements.length == 0) {
// If arrayElements is empty which means rightExpr is an empty array,
// it is technically more correct to return a TrueDimFiler here.
@ -112,11 +113,11 @@ public class ArrayContainsOperatorConversion extends BaseExpressionDimFilterOper
// to create an empty array with no argument, we just return null.
return null;
} else if (arrayElements.length == 1) {
return newSelectorDimFilter(leftExpr.getSimpleExtraction(), arrayElements[0]);
return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(arrayElements[0]));
} else {
final List<DimFilter> selectFilters = Arrays
.stream(arrayElements)
.map(val -> newSelectorDimFilter(leftExpr.getSimpleExtraction(), val))
.map(val -> newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(val)))
.collect(Collectors.toList());
return new AndDimFilter(selectFilters);
}

View File

@ -25,6 +25,7 @@ import org.apache.calcite.sql.SqlFunction;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.InputBindings;
@ -41,6 +42,7 @@ import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOperatorConversion
{
@ -111,7 +113,7 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera
// Evaluate the expression to take out the array elements.
// We can safely pass null if the expression is literal.
ExprEval<?> exprEval = expr.eval(InputBindings.nilBindings());
String[] arrayElements = exprEval.asStringArray();
Object[] arrayElements = exprEval.asArray();
if (arrayElements == null || arrayElements.length == 0) {
// If arrayElements is empty which means complexExpr is an empty array,
// it is technically more correct to return a TrueDimFiler here.
@ -119,11 +121,11 @@ public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOpera
// to create an empty array with no argument, we just return null.
return null;
} else if (arrayElements.length == 1) {
return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), arrayElements[0]);
return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), Evals.asString(arrayElements[0]));
} else {
return new InDimFilter(
simpleExtractionExpr.getSimpleExtraction().getColumn(),
new InDimFilter.ValuesSet(Arrays.asList(arrayElements)),
new InDimFilter.ValuesSet(Arrays.stream(arrayElements).map(Evals::asString).collect(Collectors.toList())),
simpleExtractionExpr.getSimpleExtraction().getExtractionFn(),
null
);

View File

@ -29,6 +29,7 @@ import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.InputBindings;
import org.apache.druid.math.expr.Parser;
@ -44,7 +45,6 @@ import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
@ -338,12 +338,14 @@ public class MultiValueStringOperatorConversions
if (!expr.isLiteral()) {
return null;
}
String[] lit = expr.eval(InputBindings.nilBindings()).asStringArray();
Object[] lit = expr.eval(InputBindings.nilBindings()).asArray();
if (lit == null || lit.length == 0) {
return null;
}
HashSet<String> literals = Sets.newHashSetWithExpectedSize(lit.length);
literals.addAll(Arrays.asList(lit));
for (Object o : lit) {
literals.add(Evals.asString(o));
}
final DruidExpression.ExpressionGenerator builder = (args) -> {
final StringBuilder expressionBuilder;

View File

@ -144,22 +144,32 @@ public class DruidRexExecutor implements RexExecutor
assert exprResult.isArray();
if (SqlTypeName.NUMERIC_TYPES.contains(constExp.getType().getComponentType().getSqlTypeName())) {
if (exprResult.type().getElementType().is(ExprType.LONG)) {
List<BigDecimal> resultAsBigDecimalList = Arrays.stream(exprResult.asLongArray())
.map(BigDecimal::valueOf)
List<BigDecimal> resultAsBigDecimalList = Arrays.stream(exprResult.asArray())
.map(val -> {
final Number longVal = (Number) val;
if (longVal == null) {
return null;
}
return BigDecimal.valueOf(longVal.longValue());
})
.collect(Collectors.toList());
literal = rexBuilder.makeLiteral(resultAsBigDecimalList, constExp.getType(), true);
} else {
List<BigDecimal> resultAsBigDecimalList = Arrays.stream(exprResult.asDoubleArray()).map(
doubleVal -> {
if (Double.isNaN(doubleVal) || Double.isInfinite(doubleVal)) {
List<BigDecimal> resultAsBigDecimalList = Arrays.stream(exprResult.asArray()).map(
val -> {
final Number doubleVal = (Number) val;
if (doubleVal == null) {
return null;
}
if (Double.isNaN(doubleVal.doubleValue()) || Double.isInfinite(doubleVal.doubleValue())) {
String expression = druidExpression.getExpression();
throw new UnsupportedSQLQueryException(
"'%s' contains an element that evaluates to '%s' which is not supported in SQL. You can either cast the element in the array to bigint or char or change the expression itself",
expression,
Double.toString(doubleVal)
Double.toString(doubleVal.doubleValue())
);
}
return BigDecimal.valueOf(doubleVal);
return BigDecimal.valueOf(doubleVal.doubleValue());
}
).collect(Collectors.toList());
literal = rexBuilder.makeLiteral(resultAsBigDecimalList, constExp.getType(), true);

View File

@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite.run;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterables;
import com.google.common.primitives.Ints;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
@ -355,7 +356,8 @@ public class NativeQueryMaker implements QueryMaker
}
private static Object maybeCoerceArrayToList(Object value, boolean mustCoerce)
@VisibleForTesting
static Object maybeCoerceArrayToList(Object value, boolean mustCoerce)
{
if (value instanceof List) {
return value;
@ -366,12 +368,23 @@ public class NativeQueryMaker implements QueryMaker
} else if (value instanceof Double[]) {
return Arrays.asList((Double[]) value);
} else if (value instanceof Object[]) {
Object[] array = (Object[]) value;
ArrayList<Object> lst = new ArrayList<>(array.length);
final Object[] array = (Object[]) value;
final ArrayList<Object> lst = new ArrayList<>(array.length);
for (Object o : array) {
lst.add(maybeCoerceArrayToList(o, false));
}
return lst;
} else if (value instanceof long[]) {
return Arrays.stream((long[]) value).boxed().collect(Collectors.toList());
} else if (value instanceof double[]) {
return Arrays.stream((double[]) value).boxed().collect(Collectors.toList());
} else if (value instanceof float[]) {
final float[] array = (float[]) value;
final ArrayList<Object> lst = new ArrayList<>(array.length);
for (float f : array) {
lst.add(f);
}
return lst;
} else if (value instanceof ComparableStringArray) {
return Arrays.asList(((ComparableStringArray) value).getDelegate());
} else if (value instanceof ComparableList) {

View File

@ -178,7 +178,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringWorksLikeStringScan()
{
final String nullVal = NullHandling.replaceWithDefault() ? "[\"foo\"]" : "[null]";
final String nullVal = NullHandling.replaceWithDefault() ? "foo" : null;
testQuery(
"SELECT concat(dim3, 'foo') FROM druid.numfoo",
ImmutableList.of(
@ -195,8 +195,8 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new Object[]{"[\"afoo\",\"bfoo\"]"},
new Object[]{"[\"bfoo\",\"cfoo\"]"},
new Object[]{"[\"dfoo\"]"},
new Object[]{"[\"foo\"]"},
new Object[]{"dfoo"},
new Object[]{"foo"},
new Object[]{nullVal},
new Object[]{nullVal}
)
@ -206,7 +206,7 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
@Test
public void testMultiValueStringWorksLikeStringSelfConcatScan()
{
final String nullVal = NullHandling.replaceWithDefault() ? "[\"-lol-\"]" : "[null]";
final String nullVal = NullHandling.replaceWithDefault() ? "-lol-" : null;
testQuery(
"SELECT concat(dim3, '-lol-', dim3) FROM druid.numfoo",
ImmutableList.of(
@ -223,8 +223,8 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
ImmutableList.of(
new Object[]{"[\"a-lol-a\",\"b-lol-b\"]"},
new Object[]{"[\"b-lol-b\",\"c-lol-c\"]"},
new Object[]{"[\"d-lol-d\"]"},
new Object[]{"[\"-lol-\"]"},
new Object[]{"d-lol-d"},
new Object[]{"-lol-"},
new Object[]{nullVal},
new Object[]{nullVal}
)
@ -387,8 +387,8 @@ public class CalciteMultiValueStringQueryTest extends BaseCalciteQueryTest
.build()
),
ImmutableList.of(
new Object[]{"[\"b\"]"},
new Object[]{"[\"c\"]"},
new Object[]{"b"},
new Object[]{"c"},
new Object[]{"[]"},
new Object[]{useDefault ? NULL_STRING : "[]"},
new Object[]{NULL_STRING},

View File

@ -0,0 +1,116 @@
/*
* 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.sql.calcite.run;
import org.apache.druid.segment.data.ComparableList;
import org.apache.druid.segment.data.ComparableStringArray;
import org.junit.Assert;
import org.junit.Test;
import java.util.Arrays;
import java.util.List;
public class NativeQueryMakerTest
{
@Test
public void testCoerceStringArrays()
{
final List<String> stringList = Arrays.asList("x", "y", "z", null);
final Object[] stringArray = new Object[]{"x", "y", "z", null};
final ComparableStringArray comparableStringArray = ComparableStringArray.of(new String[]{"x", "y", "z", null});
final String[] stringArray2 = new String[]{"x", "y", "z", null};
assertCoerced(stringList, stringList, true);
assertCoerced(stringList, stringArray, true);
assertCoerced(stringList, stringArray2, true);
assertCoerced(stringList, comparableStringArray, true);
}
@Test
public void testCoerceLongArrays()
{
final List<Long> listWithNull = Arrays.asList(1L, 2L, null, 3L);
final Long[] arrayWithNull = new Long[]{1L, 2L, null, 3L};
final ComparableList<Long> comparableList = new ComparableList<>(listWithNull);
final List<Long> list = Arrays.asList(1L, 2L, 3L);
final long[] array = new long[]{1L, 2L, 3L};
assertCoerced(listWithNull, listWithNull, true);
assertCoerced(listWithNull, arrayWithNull, true);
assertCoerced(listWithNull, comparableList, true);
assertCoerced(list, list, true);
assertCoerced(list, array, true);
}
@Test
public void testCoerceDoubleArrays()
{
final List<Double> listWithNull = Arrays.asList(1.1, 2.2, null, 3.3);
final Double[] arrayWithNull = new Double[]{1.1, 2.2, null, 3.3};
final ComparableList<Double> comparableList = new ComparableList<>(listWithNull);
final List<Double> list = Arrays.asList(1.1, 2.2, 3.3);
final double[] array = new double[]{1.1, 2.2, 3.3};
assertCoerced(listWithNull, listWithNull, true);
assertCoerced(listWithNull, arrayWithNull, true);
assertCoerced(listWithNull, comparableList, true);
assertCoerced(list, list, true);
assertCoerced(list, array, true);
}
@Test
public void testCoerceFloatArrays()
{
final List<Float> listWithNull = Arrays.asList(1.1f, 2.2f, null, 3.3f);
final Float[] arrayWithNull = new Float[]{1.1f, 2.2f, null, 3.3f};
final ComparableList<Float> comparableList = new ComparableList<>(listWithNull);
final List<Float> list = Arrays.asList(1.1f, 2.2f, 3.3f);
final float[] array = new float[]{1.1f, 2.2f, 3.3f};
assertCoerced(listWithNull, listWithNull, true);
assertCoerced(listWithNull, arrayWithNull, true);
assertCoerced(listWithNull, comparableList, true);
assertCoerced(list, list, true);
assertCoerced(list, array, true);
}
@Test
public void testCoerceNestedArrays()
{
List<?> nestedList = Arrays.asList(Arrays.asList(1L, 2L, 3L), Arrays.asList(4L, 5L, 6L));
Object[] nestedArray = new Object[]{new Object[]{1L, 2L, 3L}, new Object[]{4L, 5L, 6L}};
assertCoerced(nestedList, nestedList, true);
assertCoerced(nestedList, nestedArray, true);
}
@Test
public void testMustCoerce()
{
Assert.assertNull(NativeQueryMaker.maybeCoerceArrayToList("hello", true));
}
private static void assertCoerced(Object expected, Object toCoerce, boolean mustCoerce)
{
Object coerced = NativeQueryMaker.maybeCoerceArrayToList(toCoerce, mustCoerce);
Assert.assertEquals(expected, coerced);
}
}