mirror of https://github.com/apache/druid.git
fix issues with filtering nulls on values coerced to numeric types (#14139)
* fix issues with filtering nulls on values coerced to numeric types * fix issues with 'auto' type numeric columns in default value mode * optimize variant typed columns without nested data * more tests for 'auto' type column ingestion
This commit is contained in:
parent
0a3889b192
commit
8805d8d7db
|
@ -69,6 +69,28 @@ public class Evals
|
|||
return !NullHandling.isNullOrEquivalent(x) && Boolean.parseBoolean(x);
|
||||
}
|
||||
|
||||
/**
|
||||
* Best effort try to turn a value into a boolean:
|
||||
* {@link Boolean} will be passed directly through
|
||||
* {@link String} will use {@link #asBoolean(String)}
|
||||
* {@link Long} will use {@link #asBoolean(long)}
|
||||
* {@link Number} will use {@link #asBoolean(double)}
|
||||
* everything else, including null will be false
|
||||
*/
|
||||
public static boolean objectAsBoolean(@Nullable Object val)
|
||||
{
|
||||
if (val instanceof Boolean) {
|
||||
return (Boolean) val;
|
||||
} else if (val instanceof String) {
|
||||
return Evals.asBoolean((String) val);
|
||||
} else if (val instanceof Long) {
|
||||
return Evals.asBoolean((Long) val);
|
||||
} else if (val instanceof Number) {
|
||||
return Evals.asBoolean(((Number) val).doubleValue());
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Call {@link Object#toString()} on a non-null value
|
||||
*/
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.segment.column.NullableTypeStrategy;
|
||||
import org.apache.druid.segment.column.TypeStrategies;
|
||||
import org.apache.druid.segment.column.TypeStrategy;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -353,7 +354,7 @@ public abstract class ExprEval<T>
|
|||
case STRING:
|
||||
return ExprEval.of(String.valueOf(value));
|
||||
default:
|
||||
throw new IllegalArgumentException("invalid type " + type);
|
||||
throw new IllegalArgumentException("Invalid type, cannot coerce [" + type + "] to boolean");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -367,6 +368,9 @@ public abstract class ExprEval<T>
|
|||
|
||||
public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object value)
|
||||
{
|
||||
if (ExpressionType.NESTED_DATA.equals(outputType)) {
|
||||
return new NestedDataExprEval(value);
|
||||
}
|
||||
return new ComplexExprEval(outputType, value);
|
||||
}
|
||||
|
||||
|
@ -494,7 +498,7 @@ public abstract class ExprEval<T>
|
|||
}
|
||||
|
||||
// is this cool?
|
||||
return new ComplexExprEval(ExpressionType.UNKNOWN_COMPLEX, val);
|
||||
return ofComplex(ExpressionType.UNKNOWN_COMPLEX, val);
|
||||
}
|
||||
|
||||
public static ExprEval ofType(@Nullable ExpressionType type, @Nullable Object value)
|
||||
|
@ -549,10 +553,8 @@ public abstract class ExprEval<T>
|
|||
}
|
||||
return ofDouble(null);
|
||||
case COMPLEX:
|
||||
// json isn't currently defined in druid-core, this can be reworked once
|
||||
// https://github.com/apache/druid/pull/13698 is merged (or COMPLEX<json> is promoted to a real built-in type(s)
|
||||
if ("json".equals(type.getComplexTypeName())) {
|
||||
return ofComplex(type, value);
|
||||
if (ExpressionType.NESTED_DATA.equals(type)) {
|
||||
return ofComplex(type, StructuredData.unwrap(value));
|
||||
}
|
||||
byte[] bytes = null;
|
||||
if (value instanceof String) {
|
||||
|
@ -844,8 +846,13 @@ public abstract class ExprEval<T>
|
|||
case STRING:
|
||||
return ExprEval.ofStringArray(value == null ? null : new Object[] {value.toString()});
|
||||
}
|
||||
break;
|
||||
case COMPLEX:
|
||||
if (ExpressionType.NESTED_DATA.equals(castTo)) {
|
||||
return new NestedDataExprEval(value);
|
||||
}
|
||||
}
|
||||
throw new IAE("invalid type cannot cast " + type() + " to " + castTo);
|
||||
throw invalidCast(type(), castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -918,8 +925,13 @@ public abstract class ExprEval<T>
|
|||
case STRING:
|
||||
return ExprEval.ofStringArray(value == null ? null : new Object[] {value.toString()});
|
||||
}
|
||||
break;
|
||||
case COMPLEX:
|
||||
if (ExpressionType.NESTED_DATA.equals(castTo)) {
|
||||
return new NestedDataExprEval(value);
|
||||
}
|
||||
}
|
||||
throw new IAE("invalid type cannot cast " + type() + " to " + castTo);
|
||||
throw invalidCast(type(), castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1089,8 +1101,12 @@ public abstract class ExprEval<T>
|
|||
case STRING:
|
||||
return ExprEval.ofStringArray(value == null ? null : new Object[] {value});
|
||||
}
|
||||
case COMPLEX:
|
||||
if (ExpressionType.NESTED_DATA.equals(castTo)) {
|
||||
return new NestedDataExprEval(value);
|
||||
}
|
||||
}
|
||||
throw new IAE("invalid type cannot cast " + type() + " to " + castTo);
|
||||
throw invalidCast(type(), castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1272,9 +1288,13 @@ public abstract class ExprEval<T>
|
|||
cast[i] = ExprEval.ofType(elementType(), value[i]).castTo(elementType).value();
|
||||
}
|
||||
return ExprEval.ofArray(castTo, cast);
|
||||
case COMPLEX:
|
||||
if (ExpressionType.NESTED_DATA.equals(castTo)) {
|
||||
return new NestedDataExprEval(value);
|
||||
}
|
||||
}
|
||||
|
||||
throw new IAE("invalid type cannot cast " + type() + " to " + castTo);
|
||||
throw invalidCast(type(), castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1314,7 +1334,7 @@ public abstract class ExprEval<T>
|
|||
@Override
|
||||
public boolean isNumericNull()
|
||||
{
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1340,12 +1360,11 @@ public abstract class ExprEval<T>
|
|||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object[] asArray()
|
||||
{
|
||||
return new Object[0];
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1355,10 +1374,10 @@ public abstract class ExprEval<T>
|
|||
return this;
|
||||
}
|
||||
// allow cast of unknown complex to some other complex type
|
||||
if (expressionType.getComplexTypeName() == null) {
|
||||
return new ComplexExprEval(castTo, value);
|
||||
if (expressionType.getComplexTypeName() == null && castTo.getType().equals(ExprType.COMPLEX)) {
|
||||
return ofComplex(castTo, value);
|
||||
}
|
||||
throw new IAE("invalid type cannot cast " + expressionType + " to " + castTo);
|
||||
throw invalidCast(expressionType, castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1367,4 +1386,123 @@ public abstract class ExprEval<T>
|
|||
return new ComplexExpr(expressionType, value);
|
||||
}
|
||||
}
|
||||
|
||||
private static class NestedDataExprEval extends ExprEval<Object>
|
||||
{
|
||||
@Nullable
|
||||
private Number number;
|
||||
private boolean computedNumber = false;
|
||||
|
||||
private NestedDataExprEval(@Nullable Object value)
|
||||
{
|
||||
super(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExpressionType type()
|
||||
{
|
||||
return ExpressionType.NESTED_DATA;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNumericNull()
|
||||
{
|
||||
computeNumber();
|
||||
return number == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int asInt()
|
||||
{
|
||||
computeNumber();
|
||||
if (number != null) {
|
||||
return number.intValue();
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long asLong()
|
||||
{
|
||||
computeNumber();
|
||||
if (number != null) {
|
||||
return number.longValue();
|
||||
}
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double asDouble()
|
||||
{
|
||||
computeNumber();
|
||||
if (number != null) {
|
||||
return number.doubleValue();
|
||||
}
|
||||
return 0.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean asBoolean()
|
||||
{
|
||||
Object val = StructuredData.unwrap(value);
|
||||
if (val != null) {
|
||||
return Evals.objectAsBoolean(val);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void computeNumber()
|
||||
{
|
||||
if (!computedNumber && value != null) {
|
||||
computedNumber = true;
|
||||
Object val = StructuredData.unwrap(value);
|
||||
if (val instanceof Number) {
|
||||
number = (Number) val;
|
||||
} else if (val instanceof Boolean) {
|
||||
number = Evals.asLong((Boolean) val);
|
||||
} else if (val instanceof String) {
|
||||
number = ExprEval.computeNumber((String) val);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object[] asArray()
|
||||
{
|
||||
Object val = StructuredData.unwrap(value);
|
||||
ExprEval maybeArray = ExprEval.bestEffortOf(val);
|
||||
if (maybeArray.type().isPrimitive() || maybeArray.isArray()) {
|
||||
return maybeArray.asArray();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ExprEval castTo(ExpressionType castTo)
|
||||
{
|
||||
if (ExpressionType.NESTED_DATA.equals(castTo)) {
|
||||
return this;
|
||||
}
|
||||
|
||||
Object val = StructuredData.unwrap(value);
|
||||
ExprEval bestEffortOf = ExprEval.bestEffortOf(val);
|
||||
|
||||
if (bestEffortOf.type().isPrimitive() || bestEffortOf.type().isArray()) {
|
||||
return bestEffortOf.castTo(castTo);
|
||||
}
|
||||
throw invalidCast(ExpressionType.NESTED_DATA, castTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expr toExpr()
|
||||
{
|
||||
return new ComplexExpr(ExpressionType.NESTED_DATA, value);
|
||||
}
|
||||
}
|
||||
|
||||
public static IAE invalidCast(ExpressionType fromType, ExpressionType toType)
|
||||
{
|
||||
return new IAE("Invalid type, cannot cast [" + fromType + "] to [" + toType + "]");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -51,6 +51,8 @@ public class ExpressionType extends BaseTypeSignature<ExprType>
|
|||
new ExpressionType(ExprType.ARRAY, null, LONG);
|
||||
public static final ExpressionType DOUBLE_ARRAY =
|
||||
new ExpressionType(ExprType.ARRAY, null, DOUBLE);
|
||||
public static final ExpressionType NESTED_DATA =
|
||||
ExpressionType.fromColumnTypeStrict(ColumnType.NESTED_DATA);
|
||||
public static final ExpressionType UNKNOWN_COMPLEX =
|
||||
new ExpressionType(ExprType.COMPLEX, null, null);
|
||||
|
||||
|
|
|
@ -1950,7 +1950,7 @@ public interface Function extends NamedFunction
|
|||
castTo = ExpressionType.fromString(StringUtils.toUpperCase(y.asString()));
|
||||
}
|
||||
catch (IllegalArgumentException e) {
|
||||
throw validationFailed("invalid type %s", y.asString());
|
||||
throw validationFailed("Invalid type [%s]", y.asString());
|
||||
}
|
||||
return x.castTo(castTo);
|
||||
}
|
||||
|
|
|
@ -116,8 +116,12 @@ public class VectorProcessors
|
|||
final double[] doubles = new double[maxVectorSize];
|
||||
final boolean[] nulls;
|
||||
if (constant == null) {
|
||||
nulls = new boolean[maxVectorSize];
|
||||
Arrays.fill(nulls, NullHandling.sqlCompatible());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nulls = new boolean[maxVectorSize];
|
||||
Arrays.fill(nulls, true);
|
||||
} else {
|
||||
nulls = null;
|
||||
}
|
||||
} else {
|
||||
nulls = null;
|
||||
Arrays.fill(doubles, constant);
|
||||
|
@ -149,8 +153,12 @@ public class VectorProcessors
|
|||
final long[] longs = new long[maxVectorSize];
|
||||
final boolean[] nulls;
|
||||
if (constant == null) {
|
||||
nulls = new boolean[maxVectorSize];
|
||||
Arrays.fill(nulls, NullHandling.sqlCompatible());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nulls = new boolean[maxVectorSize];
|
||||
Arrays.fill(nulls, true);
|
||||
} else {
|
||||
nulls = null;
|
||||
}
|
||||
} else {
|
||||
nulls = null;
|
||||
Arrays.fill(longs, constant);
|
||||
|
@ -721,7 +729,7 @@ public class VectorProcessors
|
|||
if (leftNull) {
|
||||
if (rightNull) {
|
||||
output[i] = 0L;
|
||||
outputNulls[i] = NullHandling.sqlCompatible();
|
||||
outputNulls[i] = true;
|
||||
return;
|
||||
}
|
||||
final boolean bool = Evals.asBoolean(rightInput[i]);
|
||||
|
@ -770,7 +778,7 @@ public class VectorProcessors
|
|||
if (leftNull) {
|
||||
if (rightNull) {
|
||||
output[i] = 0;
|
||||
outputNulls[i] = NullHandling.sqlCompatible();
|
||||
outputNulls[i] = true;
|
||||
return;
|
||||
}
|
||||
final boolean bool = Evals.asBoolean(rightInput[i]);
|
||||
|
@ -884,7 +892,7 @@ public class VectorProcessors
|
|||
if (leftNull) {
|
||||
if (rightNull) {
|
||||
output[i] = 0L;
|
||||
outputNulls[i] = NullHandling.sqlCompatible();
|
||||
outputNulls[i] = true;
|
||||
return;
|
||||
}
|
||||
final boolean bool = Evals.asBoolean(rightInput[i]);
|
||||
|
@ -933,7 +941,7 @@ public class VectorProcessors
|
|||
if (leftNull) {
|
||||
if (rightNull) {
|
||||
output[i] = 0L;
|
||||
outputNulls[i] = NullHandling.sqlCompatible();
|
||||
outputNulls[i] = true;
|
||||
return;
|
||||
}
|
||||
final boolean bool = Evals.asBoolean(rightInput[i]);
|
||||
|
@ -980,7 +988,7 @@ public class VectorProcessors
|
|||
final boolean rightNull = rightInput[i] == null;
|
||||
if (leftNull) {
|
||||
if (rightNull) {
|
||||
outputNulls[i] = NullHandling.sqlCompatible();
|
||||
outputNulls[i] = true;
|
||||
return;
|
||||
}
|
||||
final boolean bool = Evals.asBoolean((String) rightInput[i]);
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.query.expression;
|
|||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.math.expr.Expr;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
|
@ -30,7 +29,6 @@ import org.apache.druid.math.expr.ExprMacroTable;
|
|||
import org.apache.druid.math.expr.ExprType;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.NamedFunction;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.nested.NestedPathFinder;
|
||||
import org.apache.druid.segment.nested.NestedPathPart;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
|
@ -46,10 +44,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
public class NestedDataExpressions
|
||||
{
|
||||
public static final ExpressionType TYPE = Preconditions.checkNotNull(
|
||||
ExpressionType.fromColumnType(ColumnType.NESTED_DATA)
|
||||
);
|
||||
|
||||
public static class JsonObjectExprMacro implements ExprMacroTable.ExprMacro
|
||||
{
|
||||
public static final String NAME = "json_object";
|
||||
|
@ -88,7 +82,7 @@ public class NestedDataExpressions
|
|||
theMap.put(field.asString(), unwrap(value));
|
||||
}
|
||||
|
||||
return ExprEval.ofComplex(TYPE, theMap);
|
||||
return ExprEval.ofComplex(ExpressionType.NESTED_DATA, theMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -102,7 +96,7 @@ public class NestedDataExpressions
|
|||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return TYPE;
|
||||
return ExpressionType.NESTED_DATA;
|
||||
}
|
||||
}
|
||||
return new StructExpr(args);
|
||||
|
@ -213,12 +207,12 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval arg = args.get(0).eval(bindings);
|
||||
if (arg.value() == null) {
|
||||
return ExprEval.ofComplex(TYPE, null);
|
||||
return ExprEval.ofComplex(ExpressionType.NESTED_DATA, null);
|
||||
}
|
||||
if (arg.type().is(ExprType.STRING)) {
|
||||
try {
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
ExpressionType.NESTED_DATA,
|
||||
jsonMapper.readValue(arg.asString(), Object.class)
|
||||
);
|
||||
}
|
||||
|
@ -244,7 +238,7 @@ public class NestedDataExpressions
|
|||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return TYPE;
|
||||
return ExpressionType.NESTED_DATA;
|
||||
}
|
||||
}
|
||||
return new ParseJsonExpr(args);
|
||||
|
@ -288,19 +282,19 @@ public class NestedDataExpressions
|
|||
if (arg.type().is(ExprType.STRING) && arg.value() != null) {
|
||||
try {
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
ExpressionType.NESTED_DATA,
|
||||
jsonMapper.readValue(arg.asString(), Object.class)
|
||||
);
|
||||
}
|
||||
catch (JsonProcessingException e) {
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
ExpressionType.NESTED_DATA,
|
||||
null
|
||||
);
|
||||
}
|
||||
}
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
ExpressionType.NESTED_DATA,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
@ -316,7 +310,7 @@ public class NestedDataExpressions
|
|||
@Override
|
||||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
return TYPE;
|
||||
return ExpressionType.NESTED_DATA;
|
||||
}
|
||||
}
|
||||
return new ParseJsonExpr(args);
|
||||
|
@ -440,7 +434,7 @@ public class NestedDataExpressions
|
|||
{
|
||||
ExprEval input = args.get(0).eval(bindings);
|
||||
return ExprEval.ofComplex(
|
||||
TYPE,
|
||||
ExpressionType.NESTED_DATA,
|
||||
NestedPathFinder.find(unwrap(input), parts)
|
||||
);
|
||||
}
|
||||
|
@ -457,7 +451,7 @@ public class NestedDataExpressions
|
|||
public ExpressionType getOutputType(InputBindingInspector inspector)
|
||||
{
|
||||
// call all the output JSON typed
|
||||
return TYPE;
|
||||
return ExpressionType.NESTED_DATA;
|
||||
}
|
||||
}
|
||||
return new JsonQueryExpr(args);
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidDoublePredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
@ -30,7 +29,6 @@ import javax.annotation.Nullable;
|
|||
public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
private final VectorValueSelector selector;
|
||||
private final boolean canHaveNulls = !NullHandling.replaceWithDefault();
|
||||
|
||||
public DoubleVectorValueMatcher(final VectorValueSelector selector)
|
||||
{
|
||||
|
@ -40,7 +38,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(@Nullable final String value)
|
||||
{
|
||||
if (value == null && canHaveNulls) {
|
||||
if (value == null) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
|
@ -62,7 +60,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final double[] vector = selector.getDoubleVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
int numRows = 0;
|
||||
|
||||
for (int i = 0; i < mask.getSelectionSize(); i++) {
|
||||
|
@ -97,7 +95,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final double[] vector = selector.getDoubleVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidFloatPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
@ -30,7 +29,6 @@ import javax.annotation.Nullable;
|
|||
public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
private final VectorValueSelector selector;
|
||||
private final boolean canHaveNulls = !NullHandling.replaceWithDefault();
|
||||
|
||||
public FloatVectorValueMatcher(final VectorValueSelector selector)
|
||||
{
|
||||
|
@ -40,7 +38,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(@Nullable final String value)
|
||||
{
|
||||
if (value == null && canHaveNulls) {
|
||||
if (value == null) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
|
@ -62,7 +60,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final float[] vector = selector.getFloatVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
@ -98,7 +96,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final float[] vector = selector.getFloatVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
|
@ -30,7 +29,6 @@ import javax.annotation.Nullable;
|
|||
public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
||||
{
|
||||
private final VectorValueSelector selector;
|
||||
private final boolean canHaveNulls = !NullHandling.replaceWithDefault();
|
||||
|
||||
public LongVectorValueMatcher(final VectorValueSelector selector)
|
||||
{
|
||||
|
@ -40,7 +38,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
@Override
|
||||
public VectorValueMatcher makeMatcher(@Nullable final String value)
|
||||
{
|
||||
if (value == null && canHaveNulls) {
|
||||
if (value == null) {
|
||||
return makeNullValueMatcher(selector);
|
||||
}
|
||||
|
||||
|
@ -62,7 +60,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final long[] vector = selector.getLongVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
@ -98,7 +96,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory
|
|||
final long[] vector = selector.getLongVector();
|
||||
final int[] selection = match.getSelection();
|
||||
final boolean[] nulls = selector.getNullVector();
|
||||
final boolean hasNulls = canHaveNulls && nulls != null;
|
||||
final boolean hasNulls = nulls != null;
|
||||
|
||||
int numRows = 0;
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment;
|
|||
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.MutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
|
@ -406,6 +407,7 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
if (root == null || !root.isSingleType()) {
|
||||
return null;
|
||||
}
|
||||
final Object defaultValue = getDefaultValueForType(root.getTypes().getSingleType());
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
|
@ -459,11 +461,12 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
if (0 <= dimIndex && dimIndex < dims.length) {
|
||||
final StructuredData data = (StructuredData) dims[dimIndex];
|
||||
if (data != null) {
|
||||
return ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
|
||||
final Object o = ExprEval.bestEffortOf(data.getValue()).valueOrDefault();
|
||||
return o == null ? defaultValue : o;
|
||||
}
|
||||
}
|
||||
|
||||
return null;
|
||||
return defaultValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -487,7 +490,7 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
|
||||
private StructuredDataProcessor.ProcessedValue<?> processValue(ExprEval<?> eval)
|
||||
{
|
||||
final ColumnType columnType = ExpressionType.toColumnType(eval.type());
|
||||
final ExpressionType columnType = eval.type();
|
||||
int sizeEstimate;
|
||||
switch (columnType.getType()) {
|
||||
case LONG:
|
||||
|
@ -609,4 +612,19 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
|
|||
.setHasNulls(hasNulls);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static Object getDefaultValueForType(@Nullable ColumnType columnType)
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
if (columnType != null) {
|
||||
if (ColumnType.LONG.equals(columnType)) {
|
||||
return NullHandling.defaultLongValue();
|
||||
} else if (ColumnType.DOUBLE.equals(columnType)) {
|
||||
return NullHandling.defaultDoubleValue();
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -40,7 +40,7 @@ import org.apache.druid.segment.nested.ScalarDoubleColumnSerializer;
|
|||
import org.apache.druid.segment.nested.ScalarLongColumnSerializer;
|
||||
import org.apache.druid.segment.nested.ScalarStringColumnSerializer;
|
||||
import org.apache.druid.segment.nested.SortedValueDictionary;
|
||||
import org.apache.druid.segment.nested.VariantArrayColumnSerializer;
|
||||
import org.apache.druid.segment.nested.VariantColumnSerializer;
|
||||
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||
|
||||
|
@ -81,6 +81,8 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
private NestedCommonFormatColumnSerializer serializer;
|
||||
|
||||
private ColumnType logicalType;
|
||||
private boolean isVariantType = false;
|
||||
private boolean hasOnlyNulls = false;
|
||||
|
||||
public AutoTypeColumnMerger(
|
||||
String name,
|
||||
|
@ -116,6 +118,9 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
final IndexableAdapter.NestedColumnMergable mergable = closer.register(
|
||||
adapter.getNestedColumnMergeables(name)
|
||||
);
|
||||
if (mergable == null) {
|
||||
continue;
|
||||
}
|
||||
final SortedValueDictionary dimValues = mergable.getValueDictionary();
|
||||
|
||||
boolean allNulls = dimValues == null || dimValues.allNull();
|
||||
|
@ -130,45 +135,50 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
}
|
||||
}
|
||||
|
||||
// no data, we don't need to write this column
|
||||
if (mergedFields.size() == 0) {
|
||||
hasOnlyNulls = true;
|
||||
return;
|
||||
}
|
||||
|
||||
// check to see if we can specialize the serializer after merging all the adapters
|
||||
if (isSingleTypeRoot(mergedFields)) {
|
||||
logicalType = mergedFields.get(NestedPathFinder.JSON_PATH_ROOT).getSingleType();
|
||||
final FieldTypeInfo.MutableTypeSet rootTypes = mergedFields.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
final boolean rootOnly = mergedFields.size() == 1 && rootTypes != null;
|
||||
if (rootOnly && rootTypes.getSingleType() != null) {
|
||||
logicalType = rootTypes.getSingleType();
|
||||
switch (logicalType.getType()) {
|
||||
case LONG:
|
||||
final ScalarLongColumnSerializer longSerializer = new ScalarLongColumnSerializer(
|
||||
serializer = new ScalarLongColumnSerializer(
|
||||
name,
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
serializer = longSerializer;
|
||||
break;
|
||||
case DOUBLE:
|
||||
final ScalarDoubleColumnSerializer doubleSerializer = new ScalarDoubleColumnSerializer(
|
||||
serializer = new ScalarDoubleColumnSerializer(
|
||||
name,
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
serializer = doubleSerializer;
|
||||
break;
|
||||
case STRING:
|
||||
final ScalarStringColumnSerializer stringSerializer = new ScalarStringColumnSerializer(
|
||||
serializer = new ScalarStringColumnSerializer(
|
||||
name,
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
serializer = stringSerializer;
|
||||
break;
|
||||
case ARRAY:
|
||||
final VariantArrayColumnSerializer arraySerializer = new VariantArrayColumnSerializer(
|
||||
serializer = new VariantColumnSerializer(
|
||||
name,
|
||||
null,
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
serializer = arraySerializer;
|
||||
break;
|
||||
default:
|
||||
throw new ISE(
|
||||
|
@ -177,6 +187,20 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
logicalType
|
||||
);
|
||||
}
|
||||
} else if (rootOnly) {
|
||||
// mixed type column, but only root path, we can use VariantArrayColumnSerializer
|
||||
// pick the least restrictive type for the logical type
|
||||
isVariantType = true;
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
|
||||
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
|
||||
}
|
||||
serializer = new VariantColumnSerializer(
|
||||
name,
|
||||
rootTypes.getByteValue(),
|
||||
indexSpec,
|
||||
segmentWriteOutMedium,
|
||||
closer
|
||||
);
|
||||
} else {
|
||||
// all the bells and whistles
|
||||
logicalType = ColumnType.NESTED_DATA;
|
||||
|
@ -259,13 +283,6 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
}
|
||||
}
|
||||
|
||||
private static boolean isSingleTypeRoot(SortedMap<String, FieldTypeInfo.MutableTypeSet> mergedFields)
|
||||
{
|
||||
return mergedFields.size() == 1
|
||||
&& mergedFields.get(NestedPathFinder.JSON_PATH_ROOT) != null
|
||||
&& mergedFields.get(NestedPathFinder.JSON_PATH_ROOT).getSingleType() != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector convertSortedSegmentRowValuesToMergedRowValues(
|
||||
int segmentIndex,
|
||||
|
@ -290,7 +307,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
@Override
|
||||
public boolean hasOnlyNulls()
|
||||
{
|
||||
return false;
|
||||
return hasOnlyNulls;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -301,6 +318,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
|||
final NestedCommonFormatColumnPartSerde partSerde = NestedCommonFormatColumnPartSerde.serializerBuilder()
|
||||
.withLogicalType(logicalType)
|
||||
.withHasNulls(serializer.hasNulls())
|
||||
.isVariantType(isVariantType)
|
||||
.withByteOrder(ByteOrder.nativeOrder())
|
||||
.withBitmapSerdeFactory(indexSpec.getBitmapSerdeFactory())
|
||||
.withSerializer(serializer)
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.druid.segment.nested.ScalarDoubleColumnSerializer;
|
|||
import org.apache.druid.segment.nested.ScalarLongColumnSerializer;
|
||||
import org.apache.druid.segment.nested.ScalarStringColumnSerializer;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.segment.nested.VariantArrayColumnSerializer;
|
||||
import org.apache.druid.segment.nested.VariantColumnSerializer;
|
||||
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
||||
|
||||
/**
|
||||
|
@ -50,7 +50,7 @@ import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
|||
* @see AutoTypeColumnIndexer
|
||||
* @see AutoTypeColumnMerger
|
||||
* @see NestedCommonFormatColumnSerializer
|
||||
* @see VariantArrayColumnSerializer
|
||||
* @see VariantColumnSerializer
|
||||
* @see ScalarDoubleColumnSerializer
|
||||
* @see ScalarLongColumnSerializer
|
||||
* @see NestedDataColumnSerializer
|
||||
|
|
|
@ -117,7 +117,7 @@ public class ColumnBuilder
|
|||
return this;
|
||||
}
|
||||
|
||||
public ColumnBuilder setStandardTypeColumnSupplier(Supplier<? extends NestedCommonFormatColumn> columnSupplier)
|
||||
public ColumnBuilder setNestedCommonFormatColumnSupplier(Supplier<? extends NestedCommonFormatColumn> columnSupplier)
|
||||
{
|
||||
checkColumnSupplierNotSet();
|
||||
this.columnSupplier = columnSupplier;
|
||||
|
|
|
@ -23,9 +23,11 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.annotation.JsonSerialize;
|
||||
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Native Druid types.
|
||||
|
@ -68,24 +70,28 @@ public class ColumnType extends BaseTypeSignature<ValueType>
|
|||
// currently, arrays only come from expressions or aggregators
|
||||
/**
|
||||
* An array of Strings. Values will be represented as Object[]
|
||||
*
|
||||
* @see ValueType#ARRAY
|
||||
* @see ValueType#STRING
|
||||
*/
|
||||
public static final ColumnType STRING_ARRAY = ofArray(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 = ofArray(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 = ofArray(DOUBLE);
|
||||
/**
|
||||
* An array of Floats. Values will be represented as Object[] or float[].
|
||||
*
|
||||
* @see ValueType#ARRAY
|
||||
* @see ValueType#FLOAT
|
||||
*/
|
||||
|
@ -128,4 +134,80 @@ public class ColumnType extends BaseTypeSignature<ValueType>
|
|||
{
|
||||
return ColumnTypeFactory.getInstance().ofComplex(complexTypeName);
|
||||
}
|
||||
|
||||
public static ColumnType leastRestrictiveType(@Nullable ColumnType type, @Nullable ColumnType other)
|
||||
{
|
||||
if (type == null) {
|
||||
return other;
|
||||
}
|
||||
if (other == null) {
|
||||
return type;
|
||||
}
|
||||
if (type.is(ValueType.COMPLEX) && other.is(ValueType.COMPLEX)) {
|
||||
if (type.getComplexTypeName() == null) {
|
||||
return other;
|
||||
}
|
||||
if (other.getComplexTypeName() == null) {
|
||||
return type;
|
||||
}
|
||||
if (!Objects.equals(type, other)) {
|
||||
throw new IAE("Cannot implicitly cast %s to %s", type, other);
|
||||
}
|
||||
return type;
|
||||
}
|
||||
// if either is nested data, use nested data, otherwise error
|
||||
if (type.is(ValueType.COMPLEX) || other.is(ValueType.COMPLEX)) {
|
||||
if (ColumnType.NESTED_DATA.equals(type) || ColumnType.NESTED_DATA.equals(other)) {
|
||||
return ColumnType.NESTED_DATA;
|
||||
}
|
||||
throw new IAE("Cannot implicitly cast %s to %s", type, other);
|
||||
}
|
||||
|
||||
// arrays convert based on least restrictive element type
|
||||
if (type.isArray()) {
|
||||
if (other.equals(type.getElementType())) {
|
||||
return type;
|
||||
}
|
||||
final ColumnType commonElementType;
|
||||
if (other.isArray()) {
|
||||
commonElementType = leastRestrictiveType(
|
||||
(ColumnType) type.getElementType(),
|
||||
(ColumnType) other.getElementType()
|
||||
);
|
||||
return ColumnType.ofArray(commonElementType);
|
||||
} else {
|
||||
commonElementType = leastRestrictiveType(
|
||||
(ColumnType) type.getElementType(),
|
||||
other
|
||||
);
|
||||
}
|
||||
return ColumnType.ofArray(commonElementType);
|
||||
}
|
||||
if (other.isArray()) {
|
||||
if (type.equals(type.getElementType())) {
|
||||
return type;
|
||||
}
|
||||
final ColumnType commonElementType;
|
||||
|
||||
commonElementType = leastRestrictiveType(
|
||||
type,
|
||||
(ColumnType) other.getElementType()
|
||||
);
|
||||
return ColumnType.ofArray(commonElementType);
|
||||
}
|
||||
// if either argument is a string, type becomes a string
|
||||
if (Types.is(type, ValueType.STRING) || Types.is(other, ValueType.STRING)) {
|
||||
return ColumnType.STRING;
|
||||
}
|
||||
|
||||
// all numbers win over longs
|
||||
// floats vs doubles would be handled here, but we currently only support doubles...
|
||||
if (Types.is(type, ValueType.LONG) && Types.isNullOr(other, ValueType.LONG)) {
|
||||
return ColumnType.LONG;
|
||||
}
|
||||
if (Types.is(type, ValueType.FLOAT) && Types.isNullOr(other, ValueType.FLOAT)) {
|
||||
return ColumnType.FLOAT;
|
||||
}
|
||||
return ColumnType.DOUBLE;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -137,6 +137,15 @@ public class RowSignature implements ColumnInspector
|
|||
return Optional.ofNullable(columnTypes.get(getColumnName(columnNumber)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the column is a numeric type ({@link ColumnType#isNumeric()}), otherwise false if the column
|
||||
* is not a numeric type or is not present in the row signature.
|
||||
*/
|
||||
public boolean isNumeric(final String columnName)
|
||||
{
|
||||
return getColumnType(columnName).map(ColumnType::isNumeric).orElse(false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of column names in the order they appear in this signature.
|
||||
*/
|
||||
|
|
|
@ -681,7 +681,6 @@ public class StringDictionaryEncodedColumn implements DictionaryEncodedColumn<St
|
|||
}
|
||||
|
||||
@Override
|
||||
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
|
|
|
@ -54,9 +54,12 @@ import java.util.BitSet;
|
|||
* {@link DictionaryEncodedColumn<String>} for a column which uses a {@link FrontCodedIndexed} to store its value
|
||||
* dictionary, which 'delta encodes' strings (instead of {@link org.apache.druid.segment.data.GenericIndexed} like
|
||||
* {@link StringDictionaryEncodedColumn}).
|
||||
*
|
||||
* <p>
|
||||
* This class is otherwise nearly identical to {@link StringDictionaryEncodedColumn} other than the dictionary
|
||||
* difference.
|
||||
* <p>
|
||||
* Implements {@link NestedCommonFormatColumn} so it can be used as a reader for single value string specializations
|
||||
* of {@link org.apache.druid.segment.AutoTypeColumnIndexer}.
|
||||
*/
|
||||
public class StringFrontCodedDictionaryEncodedColumn implements DictionaryEncodedColumn<String>,
|
||||
NestedCommonFormatColumn
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.List;
|
|||
|
||||
public class ComparableList<T extends Comparable> implements Comparable<ComparableList>
|
||||
{
|
||||
|
||||
private final List<T> delegate;
|
||||
|
||||
public ComparableList(List<T> input)
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
|
@ -106,20 +107,27 @@ public class BoundFilter implements Filter
|
|||
}
|
||||
final NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class);
|
||||
if (rangeIndex != null) {
|
||||
final Number lower = boundDimFilter.hasLowerBound() ? Double.parseDouble(boundDimFilter.getLower()) : null;
|
||||
final Number upper = boundDimFilter.hasUpperBound() ? Double.parseDouble(boundDimFilter.getUpper()) : null;
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
|
||||
lower,
|
||||
boundDimFilter.isLowerStrict(),
|
||||
upper,
|
||||
boundDimFilter.isUpperStrict()
|
||||
);
|
||||
if (rangeBitmaps != null) {
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
|
||||
return rangeBitmaps;
|
||||
} else {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
final Number lower = boundDimFilter.hasLowerBound() ? Doubles.tryParse(boundDimFilter.getLower()) : null;
|
||||
final Number upper = boundDimFilter.hasUpperBound() ? Doubles.tryParse(boundDimFilter.getUpper()) : null;
|
||||
// valid number bounds are required to use the range index, otherwise we need to fall back to the predicate
|
||||
// index to get consistent behavior with the value matcher. in a better world this might be a much earlier
|
||||
// validation error, but.. the bound filter doesn't live in that world
|
||||
final boolean lowerValid = !(boundDimFilter.hasLowerBound() && lower == null);
|
||||
final boolean upperValid = !(boundDimFilter.hasUpperBound() && upper == null);
|
||||
if (lowerValid && upperValid) {
|
||||
final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange(
|
||||
lower,
|
||||
boundDimFilter.isLowerStrict(),
|
||||
upper,
|
||||
boundDimFilter.isUpperStrict()
|
||||
);
|
||||
if (rangeBitmaps != null) {
|
||||
// preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set
|
||||
if (boundDimFilter.hasLowerBound() && !NullHandling.isNullOrEquivalent(boundDimFilter.getLower())) {
|
||||
return rangeBitmaps;
|
||||
} else {
|
||||
return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -127,7 +135,6 @@ public class BoundFilter implements Filter
|
|||
|
||||
// fall back to predicate based index if it is available
|
||||
return Filters.makePredicateIndex(boundDimFilter.getDimension(), selector, getPredicateFactory());
|
||||
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -136,11 +143,15 @@ public class BoundFilter implements Filter
|
|||
BitmapColumnIndex rangeIndex
|
||||
)
|
||||
{
|
||||
|
||||
|
||||
final BitmapColumnIndex nullBitmap;
|
||||
final NullValueIndex nulls = indexSupplier.as(NullValueIndex.class);
|
||||
if (nulls == null) {
|
||||
return null;
|
||||
}
|
||||
final BitmapColumnIndex nullBitmap = nulls.forNull();
|
||||
nullBitmap = nulls.forNull();
|
||||
|
||||
return new BitmapColumnIndex()
|
||||
{
|
||||
@Override
|
||||
|
|
|
@ -123,6 +123,12 @@ public class ExpressionFilter implements Filter
|
|||
ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr)
|
||||
).makeMatcher(predicateFactory);
|
||||
default:
|
||||
if (ExpressionType.NESTED_DATA.equals(outputType)) {
|
||||
return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor(
|
||||
ColumnCapabilitiesImpl.createDefault().setType(ExpressionType.toColumnType(outputType)).setHasNulls(true),
|
||||
ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr)
|
||||
).makeMatcher(predicateFactory);
|
||||
}
|
||||
throw new UOE("Vectorized expression matchers not implemented for type: [%s]", outputType);
|
||||
}
|
||||
}
|
||||
|
@ -303,6 +309,12 @@ public class ExpressionFilter implements Filter
|
|||
return Evals::asBoolean;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Object> makeObjectPredicate()
|
||||
{
|
||||
return Evals::objectAsBoolean;
|
||||
}
|
||||
|
||||
// The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining..
|
||||
// DruidPredicateFactory currently doesn't really need equals or hashcode since 'toString' method that is actually
|
||||
// called when testing equality of DimensionPredicateFilter, so it's the truly required method, but that seems
|
||||
|
|
|
@ -89,39 +89,39 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||
|
||||
/**
|
||||
* Implementation of {@link NestedDataComplexColumn} which uses a {@link CompressedVariableSizedBlobColumn} for the
|
||||
* 'raw' {@link StructuredData} values and provides selectors for nested 'literal' field columns.
|
||||
* 'raw' {@link StructuredData} values and provides selectors for nested field columns specified by ordered lists of
|
||||
* {@link NestedPathPart}.
|
||||
* <p>
|
||||
* The list of available nested paths is stored in {@link #fields}, and their associated types stored in
|
||||
* {@link #fieldInfo} which can be accessed by the index of the field in {@link #fields}.
|
||||
* <p>
|
||||
* In the case that the nested column has only a single field, and that field is the 'root' path, specified by
|
||||
* {@link #rootFieldPath}, the selectors created for the complex column itself will use the 'root' path selectors
|
||||
* instead.
|
||||
*/
|
||||
public abstract class CompressedNestedDataComplexColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||
extends NestedDataComplexColumn implements NestedCommonFormatColumn
|
||||
{
|
||||
private static final ObjectStrategy<Object> STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy();
|
||||
public static final IntTypeStrategy INT_TYPE_STRATEGY = new IntTypeStrategy();
|
||||
private final ColumnConfig columnConfig;
|
||||
private final Closer closer;
|
||||
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
||||
private CompressedVariableSizedBlobColumn compressedRawColumn;
|
||||
private final ImmutableBitmap nullValues;
|
||||
|
||||
private final GenericIndexed<String> fields;
|
||||
private final FieldTypeInfo fieldInfo;
|
||||
|
||||
private final Supplier<TStringDictionary> stringDictionarySupplier;
|
||||
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||
|
||||
private final SmooshedFileMapper fileMapper;
|
||||
|
||||
private final String rootFieldPath;
|
||||
|
||||
private final ConcurrentHashMap<Integer, ColumnHolder> columns = new ConcurrentHashMap<>();
|
||||
|
||||
private static final ObjectStrategy<Object> STRATEGY = NestedDataComplexTypeSerde.INSTANCE.getObjectStrategy();
|
||||
|
||||
private final ColumnType logicalType;
|
||||
|
||||
private final String columnName;
|
||||
private final BitmapSerdeFactory bitmapSerdeFactory;
|
||||
private final ByteOrder byteOrder;
|
||||
private final ConcurrentHashMap<Integer, ColumnHolder> columns = new ConcurrentHashMap<>();
|
||||
private CompressedVariableSizedBlobColumn compressedRawColumn;
|
||||
|
||||
public CompressedNestedDataComplexColumn(
|
||||
String columnName,
|
||||
|
@ -651,7 +651,8 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
|
|||
if (maybeArray instanceof Object[]) {
|
||||
Object[] anArray = (Object[]) maybeArray;
|
||||
if (elementNumber < anArray.length) {
|
||||
elements[i] = anArray[elementNumber];
|
||||
final Object element = anArray[elementNumber];
|
||||
elements[i] = element;
|
||||
} else {
|
||||
elements[i] = null;
|
||||
}
|
||||
|
|
|
@ -50,16 +50,23 @@ import java.nio.ByteOrder;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Base class for writer of global dictionary encoded nested literal columns for {@link NestedDataColumnSerializerV4} and
|
||||
* {@link NestedDataColumnSerializer}. While processing the 'raw' nested data, the
|
||||
* serializers will call {@link #addValue(int, Object)} for writers, which for this type of writer entails building a
|
||||
* local dictionary to map into to the global dictionary ({@link #localDictionary}) and writes this unsorted localId to
|
||||
* an intermediate integer column, {@link #intermediateValueWriter}.
|
||||
* Base class for writer of global dictionary encoded nested field columns for {@link NestedDataColumnSerializerV4} and
|
||||
* {@link NestedDataColumnSerializer}. Nested columns are written in multiple passes. The first pass processes the
|
||||
* 'raw' nested data with a {@link StructuredDataProcessor} which will call {@link #addValue(int, Object)} for writers
|
||||
* of each field which is present. For this type of writer, this entails building a local dictionary
|
||||
* ({@link #localDictionary})to map into to the global dictionary ({@link #globalDictionaryIdLookup}) and writes this
|
||||
* unsorted localId to an intermediate integer column, {@link #intermediateValueWriter}.
|
||||
* <p>
|
||||
* When processing the 'raw' value column is complete, the {@link #writeTo(int, FileSmoosher)} method will sort the
|
||||
* local ids and write them out to a local sorted dictionary, iterate over {@link #intermediateValueWriter} swapping
|
||||
* the unsorted local ids with the sorted ids and writing to the compressed id column writer
|
||||
* {@link #encodedValueSerializer} building the bitmap indexes along the way.
|
||||
* {@link #encodedValueSerializer}, building the bitmap indexes along the way.
|
||||
*
|
||||
* @see ScalarDoubleFieldColumnWriter - single type double columns
|
||||
* @see ScalarLongFieldColumnWriter - single type long columns
|
||||
* @see ScalarStringFieldColumnWriter - single type string columns
|
||||
* @see VariantArrayFieldColumnWriter - single type array columns of double, long, or string
|
||||
* @see VariantFieldColumnWriter - mixed type columns of any combination
|
||||
*/
|
||||
public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
|
||||
{
|
||||
|
@ -98,7 +105,8 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
|
|||
}
|
||||
|
||||
/**
|
||||
* Perform any value conversion needed before storing the value in the
|
||||
* Perform any value conversion needed before looking up the global id in the value dictionary (such as null handling
|
||||
* stuff or array processing to add the elements to the dictionary before adding the int[] to the dictionary)
|
||||
*/
|
||||
T processValue(int row, Object value)
|
||||
{
|
||||
|
@ -152,6 +160,9 @@ public abstract class GlobalDictionaryEncodedFieldColumnWriter<T>
|
|||
cursorPosition++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Backfill intermediate column with null values
|
||||
*/
|
||||
private void fillNull(int row) throws IOException
|
||||
{
|
||||
final T value = processValue(row, null);
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
|||
import org.apache.druid.segment.column.ColumnFormat;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.SortedMap;
|
||||
|
@ -39,7 +40,15 @@ import java.util.TreeMap;
|
|||
/**
|
||||
* Base implementation for columns created with {@link AutoTypeColumnSchema} and handled with
|
||||
* {@link NestedCommonFormatColumnHandler} to allow ease of merge via
|
||||
* {@link AutoTypeColumnMerger}
|
||||
* {@link AutoTypeColumnMerger} by providing a common implementation. All columns are read with
|
||||
* {@link NestedCommonFormatColumnPartSerde}
|
||||
*
|
||||
* @see ScalarDoubleColumn
|
||||
* @see ScalarLongColumn
|
||||
* @see ScalarStringDictionaryEncodedColumn
|
||||
* @see org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn
|
||||
* @see VariantColumn
|
||||
* @see CompressedNestedDataComplexColumn
|
||||
*/
|
||||
public interface NestedCommonFormatColumn extends BaseColumn
|
||||
{
|
||||
|
|
|
@ -32,6 +32,25 @@ import java.nio.ByteOrder;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
import java.util.SortedMap;
|
||||
|
||||
/**
|
||||
* Basic serializer implementation for the {@link NestedCommonFormatColumn} family of columns. The
|
||||
* {@link org.apache.druid.segment.AutoTypeColumnIndexer} catalogs the types and fields present in the data it processes
|
||||
* using a {@link StructuredDataProcessor}. When persisting and merging segments, the
|
||||
* {@link org.apache.druid.segment.AutoTypeColumnMerger} will choose the most appropriate serializer based on the data
|
||||
* which was processed as follows:
|
||||
*
|
||||
* @see ScalarDoubleColumnSerializer - single type double columns
|
||||
* @see ScalarLongColumnSerializer - single type long columns
|
||||
* @see ScalarStringColumnSerializer - single type string columns
|
||||
* @see VariantColumnSerializer - single type array columns of string, long, double, and mixed type columns
|
||||
*
|
||||
* @see NestedDataColumnSerializer - nested columns
|
||||
*
|
||||
* @see NestedDataColumnSerializerV4 - legacy nested column format created by
|
||||
* {@link org.apache.druid.segment.NestedDataColumnIndexer} and
|
||||
* {@link org.apache.druid.segment.NestedDataColumnMerger}
|
||||
*
|
||||
*/
|
||||
public abstract class NestedCommonFormatColumnSerializer implements GenericColumnSerializer<StructuredData>
|
||||
{
|
||||
public static final byte V0 = 0x00;
|
||||
|
@ -95,6 +114,9 @@ public abstract class NestedCommonFormatColumnSerializer implements GenericColum
|
|||
return buffer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Nested field columns are stored in separate
|
||||
*/
|
||||
public static String getInternalFileName(String fileNameBase, String field)
|
||||
{
|
||||
return StringUtils.format("%s.%s", fileNameBase, field);
|
||||
|
|
|
@ -59,6 +59,24 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.SortedMap;
|
||||
|
||||
/**
|
||||
* Serializer for {@link NestedCommonFormatColumn} which can store nested data. The serializer stores several components
|
||||
* including:
|
||||
* - a field list and associated type info
|
||||
* - value dictionaries for string, long, double, and array values (where the arrays are stored as int[] that point to
|
||||
* the string, long, and double values)
|
||||
* - raw data is stored with a {@link CompressedVariableSizedBlobColumnSerializer} as blobs of SMILE encoded data
|
||||
* - a null value bitmap to track which 'raw' rows are null
|
||||
*
|
||||
* For each nested field, a {@link GlobalDictionaryEncodedFieldColumnWriter} will write a sub-column to specialize
|
||||
* fast reading and filtering of that path.
|
||||
*
|
||||
* @see ScalarDoubleFieldColumnWriter - single type double field
|
||||
* @see ScalarLongFieldColumnWriter - single type long field
|
||||
* @see ScalarStringFieldColumnWriter - single type string field
|
||||
* @see VariantArrayFieldColumnWriter - single type array of string, long, and double field
|
||||
* @see VariantFieldColumnWriter - mixed type field
|
||||
*/
|
||||
public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
{
|
||||
private static final Logger log = new Logger(NestedDataColumnSerializer.class);
|
||||
|
|
|
@ -39,7 +39,7 @@ import java.util.List;
|
|||
* Nested data column with optimized support for simple arrays. Not actually v5 in the segment since columns are now
|
||||
* serialized using {@link org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde} instead of the generic
|
||||
* complex type system.
|
||||
*
|
||||
* <p>
|
||||
* Not really stored in a segment as V5 since instead of V5 we migrated to {@link NestedCommonFormatColumn} which
|
||||
* specializes physical format based on the types of data encountered during processing, and so versions are now
|
||||
* {@link NestedCommonFormatColumnSerializer#V0} for all associated specializations.
|
||||
|
|
|
@ -39,14 +39,14 @@ import java.util.Set;
|
|||
|
||||
/**
|
||||
* Describes the basic shape for any 'nested data' ({@link StructuredData}) {@link ComplexColumn} implementation along
|
||||
* with basic facilities for caching any columns created and methods for retrieving selectors for nested literal field
|
||||
* columns.
|
||||
* with basic facilities for caching any columns created and methods for retrieving selectors for nested field columns.
|
||||
* <p>
|
||||
* {@link org.apache.druid.segment.virtual.NestedFieldVirtualColumn} allows query time use of the nested fields.
|
||||
*/
|
||||
public abstract class NestedDataComplexColumn implements ComplexColumn
|
||||
{
|
||||
/**
|
||||
* Make a {@link DimensionSelector} for a nested literal field column associated with this nested
|
||||
* complex column specified by a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link DimensionSelector} for a nested field column
|
||||
*/
|
||||
public abstract DimensionSelector makeDimensionSelector(
|
||||
List<NestedPathPart> path,
|
||||
|
@ -55,8 +55,7 @@ public abstract class NestedDataComplexColumn implements ComplexColumn
|
|||
);
|
||||
|
||||
/**
|
||||
* Make a {@link ColumnValueSelector} for a nested literal field column associated with this nested
|
||||
* complex column specified by a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link ColumnValueSelector} for a nested field column
|
||||
*/
|
||||
public abstract ColumnValueSelector<?> makeColumnValueSelector(
|
||||
List<NestedPathPart> path,
|
||||
|
@ -64,8 +63,7 @@ public abstract class NestedDataComplexColumn implements ComplexColumn
|
|||
);
|
||||
|
||||
/**
|
||||
* Make a {@link SingleValueDimensionVectorSelector} for a nested literal field column associated with this nested
|
||||
* complex column specified by a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link SingleValueDimensionVectorSelector} for a nested field column
|
||||
*/
|
||||
public abstract SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(
|
||||
List<NestedPathPart> path,
|
||||
|
@ -73,8 +71,7 @@ public abstract class NestedDataComplexColumn implements ComplexColumn
|
|||
);
|
||||
|
||||
/**
|
||||
* Make a {@link VectorObjectSelector} for a nested literal field column associated with this nested
|
||||
* complex column located at the 'path' represented as a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link VectorObjectSelector} for a nested field column
|
||||
*/
|
||||
public abstract VectorObjectSelector makeVectorObjectSelector(
|
||||
List<NestedPathPart> path,
|
||||
|
@ -82,29 +79,40 @@ public abstract class NestedDataComplexColumn implements ComplexColumn
|
|||
);
|
||||
|
||||
/**
|
||||
* Make a {@link VectorValueSelector} for a nested literal field column associated with this nested
|
||||
* complex column located at the 'path' represented as a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link VectorValueSelector} for a nested field column
|
||||
*/
|
||||
public abstract VectorValueSelector makeVectorValueSelector(
|
||||
List<NestedPathPart> path,
|
||||
ReadableVectorOffset readableOffset
|
||||
);
|
||||
|
||||
/**
|
||||
* Get list of fields represented as a sequence of {@link NestedPathPart}
|
||||
*/
|
||||
public abstract List<List<NestedPathPart>> getNestedFields();
|
||||
|
||||
/**
|
||||
* Get all {@link ColumnType} for the nested field column
|
||||
*/
|
||||
@Nullable
|
||||
public abstract Set<ColumnType> getColumnTypes(List<NestedPathPart> path);
|
||||
|
||||
/**
|
||||
* Get a {@link ColumnHolder} for a nested field column to retrieve metadata, the column itself, or indexes.
|
||||
*/
|
||||
@Nullable
|
||||
public abstract ColumnHolder getColumnHolder(List<NestedPathPart> path);
|
||||
|
||||
/**
|
||||
* Make a {@link ColumnIndexSupplier} for a nested literal field column associated with this nested
|
||||
* complex column located at the 'path' represented as a sequence of {@link NestedPathPart}.
|
||||
* Make a {@link ColumnIndexSupplier} for a nested field column
|
||||
*/
|
||||
@Nullable
|
||||
public abstract ColumnIndexSupplier getColumnIndexSupplier(List<NestedPathPart> path);
|
||||
|
||||
/**
|
||||
* Shortcut to check if a nested field column is {@link ColumnType#isNumeric()}, useful when broadly choosing the
|
||||
* type of vector selector to be used when dealing with the path
|
||||
*/
|
||||
public abstract boolean isNumeric(List<NestedPathPart> path);
|
||||
|
||||
@Override
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DoubleColumnSelector;
|
||||
|
@ -37,6 +38,9 @@ import org.roaringbitmap.PeekableIntIterator;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* {@link NestedCommonFormatColumn} for {@link ColumnType#DOUBLE}
|
||||
*/
|
||||
public class ScalarDoubleColumn implements NestedCommonFormatColumn
|
||||
{
|
||||
private final FixedIndexed<Double> doubleDictionary;
|
||||
|
@ -91,6 +95,9 @@ public class ScalarDoubleColumn implements NestedCommonFormatColumn
|
|||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
return false;
|
||||
}
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
|
@ -134,6 +141,9 @@ public class ScalarDoubleColumn implements NestedCommonFormatColumn
|
|||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
return null;
|
||||
}
|
||||
computeVectorsIfNeeded();
|
||||
return nullVector;
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.segment.nested;
|
|||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import it.unimi.dsi.fastutil.doubles.DoubleArraySet;
|
||||
import it.unimi.dsi.fastutil.doubles.DoubleIterator;
|
||||
|
@ -29,6 +30,7 @@ import it.unimi.dsi.fastutil.ints.IntIntPair;
|
|||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
|
@ -138,8 +140,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||
|
||||
private final Supplier<ColumnarDoubles> valueColumnSupplier;
|
||||
|
@ -184,7 +184,12 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (clazz.equals(NullValueIndex.class)) {
|
||||
final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap);
|
||||
final BitmapColumnIndex nullIndex;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
} else {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap);
|
||||
}
|
||||
return (T) (NullValueIndex) () -> nullIndex;
|
||||
} else if (clazz.equals(DictionaryEncodedStringValueIndex.class)
|
||||
|| clazz.equals(DictionaryEncodedValueIndex.class)) {
|
||||
|
@ -217,20 +222,27 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
{
|
||||
final boolean inputNull = value == null;
|
||||
final Double doubleValue = Strings.isNullOrEmpty(value) ? null : Doubles.tryParse(value);
|
||||
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
|
||||
int defaultValueIndex = dictionary.indexOf(NullHandling.defaultDoubleValue());
|
||||
|
||||
return new SimpleBitmapColumnIndex()
|
||||
{
|
||||
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
|
||||
|
||||
@Override
|
||||
public double estimateSelectivity(int totalRows)
|
||||
{
|
||||
if (doubleValue == null) {
|
||||
if (inputNull) {
|
||||
if (inputNull && NullHandling.sqlCompatible()) {
|
||||
return (double) getBitmap(0).size() / totalRows;
|
||||
} else {
|
||||
return 0.0;
|
||||
}
|
||||
}
|
||||
if (NullHandling.replaceWithDefault() && doubleValue.equals(NullHandling.defaultDoubleValue())) {
|
||||
if (defaultValueIndex >= 0) {
|
||||
return ((double) getBitmap(0).size() + (double) getBitmap(defaultValueIndex).size()) / totalRows;
|
||||
}
|
||||
return (double) getBitmap(0).size() / totalRows;
|
||||
}
|
||||
final int id = dictionary.indexOf(doubleValue);
|
||||
if (id < 0) {
|
||||
return 0.0;
|
||||
|
@ -242,12 +254,24 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
|
||||
{
|
||||
if (doubleValue == null) {
|
||||
if (inputNull) {
|
||||
if (inputNull && NullHandling.sqlCompatible()) {
|
||||
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
|
||||
} else {
|
||||
// input was not null but not a double... no match
|
||||
return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
}
|
||||
}
|
||||
if (NullHandling.replaceWithDefault() && doubleValue.equals(NullHandling.defaultDoubleValue())) {
|
||||
if (defaultValueIndex >= 0) {
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(
|
||||
ImmutableList.of(
|
||||
getBitmap(0),
|
||||
getBitmap(defaultValueIndex)
|
||||
)
|
||||
);
|
||||
}
|
||||
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
|
||||
}
|
||||
final int id = dictionary.indexOf(doubleValue);
|
||||
if (id < 0) {
|
||||
return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
|
@ -274,6 +298,10 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
Double theValue = Doubles.tryParse(value);
|
||||
if (theValue != null) {
|
||||
doubles.add(theValue.doubleValue());
|
||||
// add null value index in default value mode
|
||||
if (NullHandling.replaceWithDefault() && theValue.equals(NullHandling.defaultDoubleValue())) {
|
||||
needNullCheck = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -429,7 +457,11 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
while (!nextSet && iterator.hasNext()) {
|
||||
Double nextValue = iterator.next();
|
||||
if (nextValue == null) {
|
||||
nextSet = doublePredicate.applyNull();
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nextSet = doublePredicate.applyNull();
|
||||
} else {
|
||||
nextSet = doublePredicate.applyDouble(NullHandling.defaultDoubleValue());
|
||||
}
|
||||
} else {
|
||||
nextSet = doublePredicate.applyDouble(nextValue);
|
||||
}
|
||||
|
|
|
@ -45,6 +45,9 @@ import java.nio.ByteBuffer;
|
|||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Serializer for a {@link ScalarDoubleColumn}
|
||||
*/
|
||||
public class ScalarDoubleColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
{
|
||||
private static final Logger log = new Logger(ScalarDoubleColumnSerializer.class);
|
||||
|
@ -170,8 +173,8 @@ public class ScalarDoubleColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
|
||||
// null is always 0
|
||||
doubleDictionaryWriter.write(null);
|
||||
// put a dummy string in there
|
||||
dictionaryIdLookup.addNumericNull();
|
||||
|
||||
for (Double value : doubles) {
|
||||
if (value == null) {
|
||||
continue;
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.nio.ByteOrder;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Literal field writer for double type nested columns of {@link NestedDataColumnSerializerV4}. In addition to the normal
|
||||
* Nested field writer for double type columns of {@link NestedDataColumnSerializerV4}. In addition to the normal
|
||||
* dictionary encoded column, this writer also writes an additional double value column with {@link #doublesSerializer},
|
||||
* which is written to during {@link #addValue}.
|
||||
*/
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.LongColumnSelector;
|
||||
|
@ -37,6 +38,9 @@ import org.roaringbitmap.PeekableIntIterator;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* {@link NestedCommonFormatColumn} for {@link ColumnType#LONG}
|
||||
*/
|
||||
public class ScalarLongColumn implements NestedCommonFormatColumn
|
||||
{
|
||||
private final FixedIndexed<Long> longDictionary;
|
||||
|
@ -92,6 +96,9 @@ public class ScalarLongColumn implements NestedCommonFormatColumn
|
|||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
return false;
|
||||
}
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
|
@ -135,6 +142,9 @@ public class ScalarLongColumn implements NestedCommonFormatColumn
|
|||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
return null;
|
||||
}
|
||||
computeVectorsIfNeeded();
|
||||
return nullVector;
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import it.unimi.dsi.fastutil.ints.IntIntPair;
|
||||
import it.unimi.dsi.fastutil.ints.IntIterator;
|
||||
import it.unimi.dsi.fastutil.longs.LongArraySet;
|
||||
|
@ -27,6 +28,7 @@ import it.unimi.dsi.fastutil.longs.LongIterator;
|
|||
import it.unimi.dsi.fastutil.longs.LongSet;
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -137,7 +139,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||
|
||||
private final Supplier<ColumnarLongs> valueColumnSupplier;
|
||||
|
@ -183,7 +184,12 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
public <T> T as(Class<T> clazz)
|
||||
{
|
||||
if (clazz.equals(NullValueIndex.class)) {
|
||||
final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap);
|
||||
final BitmapColumnIndex nullIndex;
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
} else {
|
||||
nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap);
|
||||
}
|
||||
return (T) (NullValueIndex) () -> nullIndex;
|
||||
} else if (clazz.equals(DictionaryEncodedStringValueIndex.class)
|
||||
|| clazz.equals(DictionaryEncodedValueIndex.class)) {
|
||||
|
@ -211,6 +217,9 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
|
||||
private class LongValueSetIndex implements StringValueSetIndex
|
||||
{
|
||||
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
||||
int defaultValueIndex = dictionary.indexOf(NullHandling.defaultLongValue());
|
||||
|
||||
@Override
|
||||
public BitmapColumnIndex forValue(@Nullable String value)
|
||||
{
|
||||
|
@ -218,18 +227,22 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
final Long longValue = GuavaUtils.tryParseLong(value);
|
||||
return new SimpleBitmapColumnIndex()
|
||||
{
|
||||
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
|
||||
|
||||
@Override
|
||||
public double estimateSelectivity(int totalRows)
|
||||
{
|
||||
if (longValue == null) {
|
||||
if (inputNull) {
|
||||
if (inputNull && NullHandling.sqlCompatible()) {
|
||||
return (double) getBitmap(0).size() / totalRows;
|
||||
} else {
|
||||
return 0.0;
|
||||
}
|
||||
}
|
||||
if (NullHandling.replaceWithDefault() && longValue.equals(NullHandling.defaultLongValue())) {
|
||||
if (defaultValueIndex >= 0) {
|
||||
return ((double) getBitmap(0).size() + (double) getBitmap(defaultValueIndex).size()) / totalRows;
|
||||
}
|
||||
return (double) getBitmap(0).size() / totalRows;
|
||||
}
|
||||
final int id = dictionary.indexOf(longValue);
|
||||
if (id < 0) {
|
||||
return 0.0;
|
||||
|
@ -241,12 +254,23 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
|
||||
{
|
||||
if (longValue == null) {
|
||||
if (inputNull) {
|
||||
if (inputNull && NullHandling.sqlCompatible()) {
|
||||
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
|
||||
} else {
|
||||
return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
}
|
||||
}
|
||||
if (NullHandling.replaceWithDefault() && longValue.equals(NullHandling.defaultLongValue())) {
|
||||
if (defaultValueIndex >= 0) {
|
||||
return bitmapResultFactory.unionDimensionValueBitmaps(
|
||||
ImmutableList.of(
|
||||
getBitmap(0),
|
||||
getBitmap(defaultValueIndex)
|
||||
)
|
||||
);
|
||||
}
|
||||
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
|
||||
}
|
||||
final int id = dictionary.indexOf(longValue);
|
||||
if (id < 0) {
|
||||
return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap());
|
||||
|
@ -273,6 +297,9 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
Long theValue = GuavaUtils.tryParseLong(value);
|
||||
if (theValue != null) {
|
||||
longs.add(theValue.longValue());
|
||||
if (NullHandling.replaceWithDefault() && theValue.equals(NullHandling.defaultLongValue())) {
|
||||
needNullCheck = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -429,7 +456,11 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
|
|||
while (!nextSet && iterator.hasNext()) {
|
||||
Long nextValue = iterator.next();
|
||||
if (nextValue == null) {
|
||||
nextSet = longPredicate.applyNull();
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
nextSet = longPredicate.applyNull();
|
||||
} else {
|
||||
nextSet = longPredicate.applyLong(NullHandling.defaultLongValue());
|
||||
}
|
||||
} else {
|
||||
nextSet = longPredicate.applyLong(nextValue);
|
||||
}
|
||||
|
|
|
@ -45,6 +45,9 @@ import java.nio.ByteBuffer;
|
|||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Serializer for a {@link ScalarLongColumn}
|
||||
*/
|
||||
public class ScalarLongColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
{
|
||||
private static final Logger log = new Logger(ScalarLongColumnSerializer.class);
|
||||
|
@ -173,6 +176,7 @@ public class ScalarLongColumnSerializer extends NestedCommonFormatColumnSerializ
|
|||
// null is always 0
|
||||
longDictionaryWriter.write(null);
|
||||
dictionaryIdLookup.addNumericNull();
|
||||
|
||||
for (Long value : longs) {
|
||||
if (value == null) {
|
||||
continue;
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.nio.ByteOrder;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Literal field writer for long type nested columns of {@link NestedDataColumnSerializerV4}. In addition to the normal
|
||||
* Nested field writer for long type columns of {@link NestedDataColumnSerializerV4}. In addition to the normal
|
||||
* dictionary encoded column, this writer also writes an additional long value column with {@link #longsSerializer},
|
||||
* which is written to during {@link #addValue}.
|
||||
*/
|
||||
|
|
|
@ -44,6 +44,12 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Serializer for a string {@link NestedCommonFormatColumn} that can be read with either
|
||||
* {@link ScalarStringDictionaryEncodedColumn} or
|
||||
* {@link org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn} (if written with a front-coded
|
||||
* dictionary).
|
||||
*/
|
||||
public class ScalarStringColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
{
|
||||
private static final Logger log = new Logger(ScalarStringColumnSerializer.class);
|
||||
|
|
|
@ -50,6 +50,12 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.BitSet;
|
||||
|
||||
/**
|
||||
* {@link NestedCommonFormatColumn} specialization for {@link ColumnType#STRING} with a generic buffer based utf8
|
||||
* dictionary. This is used when not using the more specific
|
||||
* {@link org.apache.druid.segment.column.StringFrontCodedDictionaryEncodedColumn}, and only supports single value
|
||||
* strings.
|
||||
*/
|
||||
public class ScalarStringDictionaryEncodedColumn<TIndexed extends Indexed<ByteBuffer>>
|
||||
implements DictionaryEncodedColumn<String>, NestedCommonFormatColumn
|
||||
{
|
||||
|
|
|
@ -28,7 +28,7 @@ import java.io.IOException;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Field writer for string type nested columns of {@link NestedDataColumnSerializerV4} and
|
||||
* Nested field writer for string type columns of {@link NestedDataColumnSerializerV4} and
|
||||
* {@link NestedDataColumnSerializer}
|
||||
*/
|
||||
public final class ScalarStringFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<String>
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.AutoTypeColumnIndexer;
|
||||
import org.apache.druid.segment.ComparatorDimensionDictionary;
|
||||
|
@ -83,6 +84,12 @@ public class ValueDictionary
|
|||
this.stringArrays = new TreeSet<>(ColumnType.STRING_ARRAY.getNullableStrategy());
|
||||
this.longArrays = new TreeSet<>(ColumnType.LONG_ARRAY.getNullableStrategy());
|
||||
this.doubleArrays = new TreeSet<>(ColumnType.DOUBLE_ARRAY.getNullableStrategy());
|
||||
|
||||
// always add default values in default value mode. they don't cost much even if they aren't used
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
longDictionary.add(NullHandling.defaultLongValue());
|
||||
doubleDictionary.add(NullHandling.defaultDoubleValue());
|
||||
}
|
||||
}
|
||||
|
||||
public int addLongValue(@Nullable Long value)
|
||||
|
|
|
@ -1,382 +0,0 @@
|
|||
/*
|
||||
* 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.segment.nested;
|
||||
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.FixedIndexed;
|
||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
import org.apache.druid.segment.vector.ReadableVectorInspector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Iterator;
|
||||
|
||||
public class VariantArrayColumn<TStringDictionary extends Indexed<ByteBuffer>> implements NestedCommonFormatColumn
|
||||
{
|
||||
private final TStringDictionary stringDictionary;
|
||||
private final FixedIndexed<Long> longDictionary;
|
||||
private final FixedIndexed<Double> doubleDictionary;
|
||||
private final FrontCodedIntArrayIndexed arrayDictionary;
|
||||
private final ColumnarInts encodedValueColumn;
|
||||
private final ImmutableBitmap nullValueBitmap;
|
||||
private final ColumnType logicalType;
|
||||
private final int adjustLongId;
|
||||
private final int adjustDoubleId;
|
||||
private final int adjustArrayId;
|
||||
|
||||
public VariantArrayColumn(
|
||||
TStringDictionary stringDictionary,
|
||||
FixedIndexed<Long> longDictionary,
|
||||
FixedIndexed<Double> doubleDictionary,
|
||||
FrontCodedIntArrayIndexed arrayDictionary,
|
||||
ColumnarInts encodedValueColumn,
|
||||
ImmutableBitmap nullValueBitmap,
|
||||
ColumnType logicalType
|
||||
)
|
||||
{
|
||||
this.stringDictionary = stringDictionary;
|
||||
this.longDictionary = longDictionary;
|
||||
this.doubleDictionary = doubleDictionary;
|
||||
this.arrayDictionary = arrayDictionary;
|
||||
this.encodedValueColumn = encodedValueColumn;
|
||||
this.nullValueBitmap = nullValueBitmap;
|
||||
this.logicalType = logicalType;
|
||||
this.adjustLongId = stringDictionary.size();
|
||||
this.adjustDoubleId = adjustLongId + longDictionary.size();
|
||||
this.adjustArrayId = adjustDoubleId + doubleDictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getLogicalType()
|
||||
{
|
||||
return logicalType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getStringDictionary()
|
||||
{
|
||||
return new StringEncodingStrategies.Utf8ToStringIndexed(stringDictionary);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Long> getLongDictionary()
|
||||
{
|
||||
return longDictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Double> getDoubleDictionary()
|
||||
{
|
||||
return doubleDictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Object[]> getArrayDictionary()
|
||||
{
|
||||
Iterable<Object[]> arrays = () -> {
|
||||
|
||||
return new Iterator<Object[]>()
|
||||
{
|
||||
final Iterator<int[]> delegate = arrayDictionary.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return delegate.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] next()
|
||||
{
|
||||
final int[] next = delegate.next();
|
||||
final Object[] nextArray = new Object[next.length];
|
||||
for (int i = 0; i < nextArray.length; i++) {
|
||||
nextArray[i] = lookupId(next[i]);
|
||||
}
|
||||
return nextArray;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Object lookupId(int globalId)
|
||||
{
|
||||
if (globalId == 0) {
|
||||
return null;
|
||||
}
|
||||
final int adjustLongId = stringDictionary.size();
|
||||
final int adjustDoubleId = stringDictionary.size() + longDictionary.size();
|
||||
if (globalId < adjustLongId) {
|
||||
return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId));
|
||||
} else if (globalId < adjustDoubleId) {
|
||||
return longDictionary.get(globalId - adjustLongId);
|
||||
} else if (globalId < adjustDoubleId + doubleDictionary.size()) {
|
||||
return doubleDictionary.get(globalId - adjustDoubleId);
|
||||
}
|
||||
throw new IAE("Unknown globalId [%s]", globalId);
|
||||
}
|
||||
};
|
||||
};
|
||||
return new Indexed<Object[]>()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return arrayDictionary.size();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object[] get(int index)
|
||||
{
|
||||
throw new UnsupportedOperationException("get not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int indexOf(@Nullable Object[] value)
|
||||
{
|
||||
throw new UnsupportedOperationException("indexOf not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Object[]> iterator()
|
||||
{
|
||||
return arrays.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
// meh
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(ReadableOffset offset)
|
||||
{
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int nullMark = -1;
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id < adjustArrayId) {
|
||||
return lookupScalarValue(id);
|
||||
} else {
|
||||
int[] arr = arrayDictionary.get(id - adjustArrayId);
|
||||
if (arr == null) {
|
||||
return null;
|
||||
}
|
||||
final Object[] array = new Object[arr.length];
|
||||
for (int i = 0; i < arr.length; i++) {
|
||||
array[i] = lookupScalarValue(arr[i]);
|
||||
}
|
||||
return array;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0f;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to float
|
||||
Float f = Floats.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return f == null ? 0f : f;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).floatValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).floatValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0.0;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to double
|
||||
Double d = Doubles.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return d == null ? 0.0 : d;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).doubleValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0L;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to long
|
||||
Long l = GuavaUtils.tryParseLong(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return l == null ? 0L : l;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId);
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).longValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
nullMark = -1;
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(offsetMark);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
if (nullMark == offsetMark) {
|
||||
return true;
|
||||
}
|
||||
return DimensionHandlerUtils.isNumericNull(getObject());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("encodedValueColumn", encodedValueColumn);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
return new VectorObjectSelector()
|
||||
{
|
||||
private final int[] vector = new int[offset.getMaxVectorSize()];
|
||||
private final Object[] objects = new Object[offset.getMaxVectorSize()];
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
@Override
|
||||
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (id == offset.getId()) {
|
||||
return objects;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
encodedValueColumn.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
encodedValueColumn.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
final int globalId = vector[i];
|
||||
if (globalId < adjustArrayId) {
|
||||
objects[i] = lookupScalarValue(globalId);
|
||||
} else {
|
||||
int[] arr = arrayDictionary.get(globalId - adjustArrayId);
|
||||
if (arr == null) {
|
||||
objects[i] = null;
|
||||
} else {
|
||||
final Object[] array = new Object[arr.length];
|
||||
for (int j = 0; j < arr.length; j++) {
|
||||
array[j] = lookupScalarValue(arr[j]);
|
||||
}
|
||||
objects[i] = array;
|
||||
}
|
||||
}
|
||||
}
|
||||
id = offset.getId();
|
||||
|
||||
return objects;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
encodedValueColumn.close();
|
||||
}
|
||||
|
||||
private Object lookupScalarValue(int globalId)
|
||||
{
|
||||
if (globalId < adjustLongId) {
|
||||
return StringUtils.fromUtf8Nullable(stringDictionary.get(globalId));
|
||||
} else if (globalId < adjustDoubleId) {
|
||||
return longDictionary.get(globalId - adjustLongId);
|
||||
} else if (globalId < adjustArrayId) {
|
||||
return doubleDictionary.get(globalId - adjustDoubleId);
|
||||
}
|
||||
throw new IllegalArgumentException("not a scalar in the dictionary");
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||
|
@ -27,6 +28,10 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
|||
import java.io.IOException;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Nested field writer for array type columns of {@link NestedDataColumnSerializer}. This is currently only used
|
||||
* for single type array columns, but can handle any type of array. {@link VariantFieldColumnWriter} is used for mixed
|
||||
*/
|
||||
public class VariantArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<int[]>
|
||||
{
|
||||
|
||||
|
@ -44,6 +49,7 @@ public class VariantArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldC
|
|||
@Override
|
||||
int[] processValue(int row, Object value)
|
||||
{
|
||||
// replace Object[] with int[] composed of the global ids
|
||||
if (value instanceof Object[]) {
|
||||
Object[] array = (Object[]) value;
|
||||
final int[] globalIds = new int[array.length];
|
||||
|
@ -67,6 +73,10 @@ public class VariantArrayFieldColumnWriter extends GlobalDictionaryEncodedFieldC
|
|||
}
|
||||
return globalIds;
|
||||
}
|
||||
if (value != null) {
|
||||
// this writer is only used for arrays so all values will have been coerced to Object[] at this point
|
||||
throw new ISE("Value is not an array, got [%s] instead", value.getClass());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,783 @@
|
|||
/*
|
||||
* 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.segment.nested;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Predicates;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import com.google.common.primitives.Floats;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.AbstractDimensionSelector;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.FixedIndexed;
|
||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
import org.apache.druid.segment.data.SingleIndexedInt;
|
||||
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||
import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector;
|
||||
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorInspector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.roaringbitmap.PeekableIntIterator;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.BitSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* {@link NestedCommonFormatColumn} for single type array columns, and mixed type columns. If {@link #variantTypes}
|
||||
* is non-null, the column is composed of all of the types defined there, otherwise all rows are consistently
|
||||
* {@link #logicalType}. If mixed type, logical type is set by {@link ColumnType#leastRestrictiveType}.
|
||||
*/
|
||||
public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
|
||||
implements DictionaryEncodedColumn<String>, NestedCommonFormatColumn
|
||||
{
|
||||
private final TStringDictionary stringDictionary;
|
||||
private final FixedIndexed<Long> longDictionary;
|
||||
private final FixedIndexed<Double> doubleDictionary;
|
||||
private final FrontCodedIntArrayIndexed arrayDictionary;
|
||||
private final ColumnarInts encodedValueColumn;
|
||||
private final ImmutableBitmap nullValueBitmap;
|
||||
private final ColumnType logicalType;
|
||||
private final ExpressionType logicalExpressionType;
|
||||
@Nullable
|
||||
private final FieldTypeInfo.TypeSet variantTypes;
|
||||
private final int adjustLongId;
|
||||
private final int adjustDoubleId;
|
||||
private final int adjustArrayId;
|
||||
|
||||
public VariantColumn(
|
||||
TStringDictionary stringDictionary,
|
||||
FixedIndexed<Long> longDictionary,
|
||||
FixedIndexed<Double> doubleDictionary,
|
||||
FrontCodedIntArrayIndexed arrayDictionary,
|
||||
ColumnarInts encodedValueColumn,
|
||||
ImmutableBitmap nullValueBitmap,
|
||||
ColumnType logicalType,
|
||||
@Nullable Byte variantTypeSetByte
|
||||
)
|
||||
{
|
||||
this.stringDictionary = stringDictionary;
|
||||
this.longDictionary = longDictionary;
|
||||
this.doubleDictionary = doubleDictionary;
|
||||
this.arrayDictionary = arrayDictionary;
|
||||
this.encodedValueColumn = encodedValueColumn;
|
||||
this.nullValueBitmap = nullValueBitmap;
|
||||
this.logicalType = logicalType;
|
||||
this.logicalExpressionType = ExpressionType.fromColumnTypeStrict(logicalType);
|
||||
this.variantTypes = variantTypeSetByte == null ? null : new FieldTypeInfo.TypeSet(variantTypeSetByte);
|
||||
this.adjustLongId = stringDictionary.size();
|
||||
this.adjustDoubleId = adjustLongId + longDictionary.size();
|
||||
this.adjustArrayId = adjustDoubleId + doubleDictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnType getLogicalType()
|
||||
{
|
||||
return logicalType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<String> getStringDictionary()
|
||||
{
|
||||
return new StringEncodingStrategies.Utf8ToStringIndexed(stringDictionary);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Long> getLongDictionary()
|
||||
{
|
||||
return longDictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Double> getDoubleDictionary()
|
||||
{
|
||||
return doubleDictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Indexed<Object[]> getArrayDictionary()
|
||||
{
|
||||
Iterable<Object[]> arrays = () -> new Iterator<Object[]>()
|
||||
{
|
||||
final Iterator<int[]> delegate = arrayDictionary.iterator();
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return delegate.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object[] next()
|
||||
{
|
||||
final int[] next = delegate.next();
|
||||
final Object[] nextArray = new Object[next.length];
|
||||
for (int i = 0; i < nextArray.length; i++) {
|
||||
nextArray[i] = lookupId(next[i]);
|
||||
}
|
||||
return nextArray;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private Object lookupId(int id)
|
||||
{
|
||||
if (id == 0) {
|
||||
return null;
|
||||
}
|
||||
final int adjustLongId = stringDictionary.size();
|
||||
final int adjustDoubleId = stringDictionary.size() + longDictionary.size();
|
||||
if (id < adjustLongId) {
|
||||
return StringUtils.fromUtf8Nullable(stringDictionary.get(id));
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId);
|
||||
} else if (id < adjustDoubleId + doubleDictionary.size()) {
|
||||
return doubleDictionary.get(id - adjustDoubleId);
|
||||
}
|
||||
throw new IAE("Unknown id [%s]", id);
|
||||
}
|
||||
};
|
||||
return new Indexed<Object[]>()
|
||||
{
|
||||
@Override
|
||||
public int size()
|
||||
{
|
||||
return arrayDictionary.size();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object[] get(int index)
|
||||
{
|
||||
throw new UnsupportedOperationException("get not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int indexOf(@Nullable Object[] value)
|
||||
{
|
||||
throw new UnsupportedOperationException("indexOf not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<Object[]> iterator()
|
||||
{
|
||||
return arrays.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
// meh
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SortedMap<String, FieldTypeInfo.MutableTypeSet> getFieldTypeInfo()
|
||||
{
|
||||
if (variantTypes != null) {
|
||||
FieldTypeInfo.MutableTypeSet rootOnlyType = new FieldTypeInfo.MutableTypeSet(variantTypes.getByteValue());
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields = new TreeMap<>();
|
||||
fields.put(NestedPathFinder.JSON_PATH_ROOT, rootOnlyType);
|
||||
return fields;
|
||||
}
|
||||
FieldTypeInfo.MutableTypeSet rootOnlyType = new FieldTypeInfo.MutableTypeSet().add(getLogicalType());
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields = new TreeMap<>();
|
||||
fields.put(NestedPathFinder.JSON_PATH_ROOT, rootOnlyType);
|
||||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length()
|
||||
{
|
||||
return encodedValueColumn.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getSingleValueRow(int rowNum)
|
||||
{
|
||||
return encodedValueColumn.get(rowNum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getMultiValueRow(int rowNum)
|
||||
{
|
||||
throw new IllegalStateException("Multi-value row not supported");
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
if (id < stringDictionary.size()) {
|
||||
return StringUtils.fromUtf8Nullable(stringDictionary.get(id));
|
||||
} else if (id < stringDictionary.size() + longDictionary.size()) {
|
||||
return String.valueOf(longDictionary.get(id - adjustLongId));
|
||||
} else if (id < stringDictionary.size() + longDictionary.size() + doubleDictionary.size()) {
|
||||
return String.valueOf(doubleDictionary.get(id - adjustDoubleId));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String val)
|
||||
{
|
||||
if (val == null) {
|
||||
return 0;
|
||||
}
|
||||
int candidate = stringDictionary.indexOf(StringUtils.toUtf8ByteBuffer(val));
|
||||
if (candidate >= 0) {
|
||||
return candidate;
|
||||
}
|
||||
candidate = longDictionary.indexOf(GuavaUtils.tryParseLong(val));
|
||||
if (candidate >= 0) {
|
||||
candidate += adjustLongId;
|
||||
return candidate;
|
||||
}
|
||||
candidate = doubleDictionary.indexOf(Doubles.tryParse(val));
|
||||
if (candidate >= 0) {
|
||||
candidate += adjustDoubleId;
|
||||
return candidate;
|
||||
}
|
||||
|
||||
// not in here, we can't really do anything cool here
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCardinality()
|
||||
{
|
||||
if (logicalType.isArray() && variantTypes == null) {
|
||||
return arrayDictionary.size();
|
||||
}
|
||||
// this probably isn't correct if we expose this as a multi-value dimension instead of an array, which would leave
|
||||
// the array dictionary out of this computation
|
||||
return stringDictionary.size() + longDictionary.size() + doubleDictionary.size() + arrayDictionary.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimensionSelector makeDimensionSelector(
|
||||
ReadableOffset offset,
|
||||
@Nullable ExtractionFn extractionFn
|
||||
)
|
||||
{
|
||||
if (logicalType.isArray()) {
|
||||
throw new IAE("Dimension selector is currently unsupported for [%s]", logicalType);
|
||||
}
|
||||
// copy everywhere all the time
|
||||
class StringDimensionSelector extends AbstractDimensionSelector
|
||||
implements SingleValueHistoricalDimensionSelector, IdLookup
|
||||
{
|
||||
private final SingleIndexedInt row = new SingleIndexedInt();
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
row.setValue(getRowValue());
|
||||
return row;
|
||||
}
|
||||
|
||||
public int getRowValue()
|
||||
{
|
||||
return encodedValueColumn.get(offset.getOffset());
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
final int id = getRowValue();
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0f;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to float
|
||||
Float f = Floats.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return f == null ? 0f : f;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).floatValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).floatValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
final int id = getRowValue();
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0.0;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to double
|
||||
Double d = Doubles.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return d == null ? 0.0 : d;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).doubleValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
final int id = getRowValue();
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0L;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to long
|
||||
Long l = GuavaUtils.tryParseLong(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return l == null ? 0L : l;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId);
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).longValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
if (getRowValue() == 0) {
|
||||
return true;
|
||||
}
|
||||
return DimensionHandlerUtils.isNumericNull(getObject());
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedInts getRow(int offset)
|
||||
{
|
||||
row.setValue(getRowValue(offset));
|
||||
return row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getRowValue(int offset)
|
||||
{
|
||||
return encodedValueColumn.get(offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final @Nullable String value)
|
||||
{
|
||||
if (extractionFn == null) {
|
||||
final int valueId = lookupId(value);
|
||||
if (valueId >= 0) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return getRowValue() == valueId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", VariantColumn.this);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return BooleanValueMatcher.of(false);
|
||||
}
|
||||
} else {
|
||||
// Employ caching BitSet optimization
|
||||
return makeValueMatcher(Predicates.equalTo(value));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||
{
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
final int id = getRowValue();
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final boolean matches = predicate.apply(lookupName(id));
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
}
|
||||
return matches;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", VariantColumn.this);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
return VariantColumn.this.lookupName(getRowValue());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class classOfObject()
|
||||
{
|
||||
return String.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("column", encodedValueColumn);
|
||||
inspector.visit("offset", offset);
|
||||
inspector.visit("extractionFn", extractionFn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return getCardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
final String value = VariantColumn.this.lookupName(id);
|
||||
return extractionFn == null ? value : extractionFn.apply(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public IdLookup idLookup()
|
||||
{
|
||||
return extractionFn == null ? this : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
if (extractionFn == null) {
|
||||
return VariantColumn.this.lookupId(name);
|
||||
}
|
||||
throw new UnsupportedOperationException("cannot perform lookup when applying an extraction function");
|
||||
}
|
||||
}
|
||||
|
||||
return new StringDimensionSelector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnValueSelector<?> makeColumnValueSelector(ReadableOffset offset)
|
||||
{
|
||||
return new ColumnValueSelector<Object>()
|
||||
{
|
||||
private PeekableIntIterator nullIterator = nullValueBitmap.peekableIterator();
|
||||
private int nullMark = -1;
|
||||
private int offsetMark = -1;
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public Object getObject()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id < adjustArrayId) {
|
||||
return lookupScalarValue(id);
|
||||
} else {
|
||||
int[] arr = arrayDictionary.get(id - adjustArrayId);
|
||||
if (arr == null) {
|
||||
return null;
|
||||
}
|
||||
final Object[] array = new Object[arr.length];
|
||||
for (int i = 0; i < arr.length; i++) {
|
||||
array[i] = lookupScalarValue(arr[i]);
|
||||
}
|
||||
return array;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0f;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to float
|
||||
Float f = Floats.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return f == null ? 0f : f;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).floatValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).floatValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getDouble()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0.0;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to double
|
||||
Double d = Doubles.tryParse(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return d == null ? 0.0 : d;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId).doubleValue();
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
final int id = encodedValueColumn.get(offset.getOffset());
|
||||
if (id == 0) {
|
||||
// zero
|
||||
return 0L;
|
||||
} else if (id < adjustLongId) {
|
||||
// try to convert string to long
|
||||
Long l = GuavaUtils.tryParseLong(StringUtils.fromUtf8(stringDictionary.get(id)));
|
||||
return l == null ? 0L : l;
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId);
|
||||
} else {
|
||||
return doubleDictionary.get(id - adjustDoubleId).longValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isNull()
|
||||
{
|
||||
final int i = offset.getOffset();
|
||||
if (i < offsetMark) {
|
||||
// offset was reset, reset iterator state
|
||||
nullMark = -1;
|
||||
nullIterator = nullValueBitmap.peekableIterator();
|
||||
}
|
||||
offsetMark = i;
|
||||
if (nullMark < i) {
|
||||
nullIterator.advanceIfNeeded(offsetMark);
|
||||
if (nullIterator.hasNext()) {
|
||||
nullMark = nullIterator.next();
|
||||
}
|
||||
}
|
||||
if (nullMark == offsetMark) {
|
||||
return true;
|
||||
}
|
||||
return DimensionHandlerUtils.isNumericNull(getObject());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Class<?> classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("encodedValueColumn", encodedValueColumn);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
final class StringVectorSelector extends StringDictionaryEncodedColumn.StringSingleValueDimensionVectorSelector
|
||||
{
|
||||
public StringVectorSelector()
|
||||
{
|
||||
super(encodedValueColumn, offset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return getCardinality();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public String lookupName(final int id)
|
||||
{
|
||||
return VariantColumn.this.lookupName(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(@Nullable String name)
|
||||
{
|
||||
return VariantColumn.this.lookupId(name);
|
||||
}
|
||||
}
|
||||
|
||||
return new StringVectorSelector();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(ReadableVectorOffset vectorOffset)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public VectorObjectSelector makeVectorObjectSelector(ReadableVectorOffset offset)
|
||||
{
|
||||
return new VectorObjectSelector()
|
||||
{
|
||||
private final int[] vector = new int[offset.getMaxVectorSize()];
|
||||
private final Object[] objects = new Object[offset.getMaxVectorSize()];
|
||||
private int offsetId = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
@Override
|
||||
|
||||
public Object[] getObjectVector()
|
||||
{
|
||||
if (offsetId == offset.getId()) {
|
||||
return objects;
|
||||
}
|
||||
|
||||
if (offset.isContiguous()) {
|
||||
encodedValueColumn.get(vector, offset.getStartOffset(), offset.getCurrentVectorSize());
|
||||
} else {
|
||||
encodedValueColumn.get(vector, offset.getOffsets(), offset.getCurrentVectorSize());
|
||||
}
|
||||
for (int i = 0; i < offset.getCurrentVectorSize(); i++) {
|
||||
final int dictionaryId = vector[i];
|
||||
if (dictionaryId < adjustArrayId) {
|
||||
objects[i] = lookupScalarValueStrict(dictionaryId);
|
||||
} else {
|
||||
int[] arr = arrayDictionary.get(dictionaryId - adjustArrayId);
|
||||
if (arr == null) {
|
||||
objects[i] = null;
|
||||
} else {
|
||||
final Object[] array = new Object[arr.length];
|
||||
for (int j = 0; j < arr.length; j++) {
|
||||
array[j] = lookupScalarValue(arr[j]);
|
||||
}
|
||||
objects[i] = ExprEval.ofType(logicalExpressionType, array).asArray();
|
||||
}
|
||||
}
|
||||
}
|
||||
offsetId = offset.getId();
|
||||
|
||||
return objects;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return offset.getMaxVectorSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
{
|
||||
return offset.getCurrentVectorSize();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
encodedValueColumn.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup value from appropriate scalar value dictionary, coercing the value to {@link #logicalType}, particularly
|
||||
* useful for the vector query engine which prefers all the types are consistent
|
||||
* <p>
|
||||
* This method should NEVER be used when values must round trip to be able to be looked up from the array value
|
||||
* dictionary since it might coerce element values to a different type
|
||||
*/
|
||||
private Object lookupScalarValueStrict(int id)
|
||||
{
|
||||
if (id == 0) {
|
||||
return null;
|
||||
}
|
||||
if (variantTypes == null) {
|
||||
return lookupScalarValue(id);
|
||||
} else {
|
||||
ExprEval eval = ExprEval.ofType(logicalExpressionType, lookupScalarValue(id));
|
||||
return eval.value();
|
||||
}
|
||||
}
|
||||
|
||||
private Object lookupScalarValue(int id)
|
||||
{
|
||||
if (id < adjustLongId) {
|
||||
return StringUtils.fromUtf8Nullable(stringDictionary.get(id));
|
||||
} else if (id < adjustDoubleId) {
|
||||
return longDictionary.get(id - adjustLongId);
|
||||
} else if (id < adjustArrayId) {
|
||||
return doubleDictionary.get(id - adjustDoubleId);
|
||||
}
|
||||
throw new IllegalArgumentException("not a scalar in the dictionary");
|
||||
}
|
||||
}
|
|
@ -50,9 +50,9 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommonFormatColumn>, ColumnIndexSupplier
|
||||
public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonFormatColumn>, ColumnIndexSupplier
|
||||
{
|
||||
public static VariantArrayColumnAndIndexSupplier read(
|
||||
public static VariantColumnAndIndexSupplier read(
|
||||
ColumnType logicalType,
|
||||
ByteOrder byteOrder,
|
||||
BitmapSerdeFactory bitmapSerdeFactory,
|
||||
|
@ -65,6 +65,17 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
final int columnNameLength = VByte.readInt(bb);
|
||||
final String columnName = StringUtils.fromUtf8(bb, columnNameLength);
|
||||
|
||||
// variant types store an extra byte containing a FieldTypeInfo.TypeSet which has bits set for all types
|
||||
// present in the varaint column. this is a smaller scale, single path version of what a full nested column stores
|
||||
// for each nested path. If this value is present then the column is a mixed type and the logical type represents
|
||||
// the 'least restrictive' native Druid type, if not then all values consistently match the logical type
|
||||
final Byte variantTypeByte;
|
||||
if (bb.hasRemaining()) {
|
||||
variantTypeByte = bb.get();
|
||||
} else {
|
||||
variantTypeByte = null;
|
||||
}
|
||||
|
||||
if (version == NestedCommonFormatColumnSerializer.V0) {
|
||||
try {
|
||||
final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
|
||||
|
@ -163,8 +174,9 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
try (ColumnarInts throwAway = ints.get()) {
|
||||
size = throwAway.size();
|
||||
}
|
||||
return new VariantArrayColumnAndIndexSupplier(
|
||||
return new VariantColumnAndIndexSupplier(
|
||||
logicalType,
|
||||
variantTypeByte,
|
||||
stringDictionary,
|
||||
frontCodedStringDictionarySupplier,
|
||||
longDictionarySupplier,
|
||||
|
@ -187,6 +199,8 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
|
||||
|
||||
private final ColumnType logicalType;
|
||||
@Nullable
|
||||
private final Byte variantTypeSetByte;
|
||||
|
||||
private final GenericIndexed<ByteBuffer> stringDictionary;
|
||||
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
||||
|
@ -198,8 +212,9 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
private final GenericIndexed<ImmutableBitmap> valueIndexes;
|
||||
private final ImmutableBitmap nullValueBitmap;
|
||||
|
||||
public VariantArrayColumnAndIndexSupplier(
|
||||
public VariantColumnAndIndexSupplier(
|
||||
ColumnType logicalType,
|
||||
@Nullable Byte variantTypeSetByte,
|
||||
GenericIndexed<ByteBuffer> stringDictionary,
|
||||
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
|
||||
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
||||
|
@ -213,6 +228,7 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
)
|
||||
{
|
||||
this.logicalType = logicalType;
|
||||
this.variantTypeSetByte = variantTypeSetByte;
|
||||
this.stringDictionary = stringDictionary;
|
||||
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
|
||||
this.longDictionarySupplier = longDictionarySupplier;
|
||||
|
@ -227,24 +243,26 @@ public class VariantArrayColumnAndIndexSupplier implements Supplier<NestedCommon
|
|||
public NestedCommonFormatColumn get()
|
||||
{
|
||||
if (frontCodedStringDictionarySupplier != null) {
|
||||
return new VariantArrayColumn<>(
|
||||
return new VariantColumn<>(
|
||||
frontCodedStringDictionarySupplier.get(),
|
||||
longDictionarySupplier.get(),
|
||||
doubleDictionarySupplier.get(),
|
||||
arrayDictionarySupplier.get(),
|
||||
encodedValueColumnSupplier.get(),
|
||||
nullValueBitmap,
|
||||
logicalType
|
||||
logicalType,
|
||||
variantTypeSetByte
|
||||
);
|
||||
}
|
||||
return new VariantArrayColumn<>(
|
||||
return new VariantColumn<>(
|
||||
stringDictionary.singleThreaded(),
|
||||
longDictionarySupplier.get(),
|
||||
doubleDictionarySupplier.get(),
|
||||
arrayDictionarySupplier.get(),
|
||||
encodedValueColumnSupplier.get(),
|
||||
nullValueBitmap,
|
||||
logicalType
|
||||
logicalType,
|
||||
variantTypeSetByte
|
||||
);
|
||||
}
|
||||
|
|
@ -45,14 +45,20 @@ import org.apache.druid.segment.data.GenericIndexedWriter;
|
|||
import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
/**
|
||||
* Serializer for a {@link NestedCommonFormatColumn} for single type arrays and mixed type columns, but not columns
|
||||
* with nested data. If {@link #variantTypeSetByte} is set then the column has mixed types, which is added to the base
|
||||
* metadata stored in the column file.
|
||||
*/
|
||||
public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||
{
|
||||
private static final Logger log = new Logger(VariantArrayColumnSerializer.class);
|
||||
private static final Logger log = new Logger(VariantColumnSerializer.class);
|
||||
|
||||
private final String name;
|
||||
private final SegmentWriteOutMedium segmentWriteOutMedium;
|
||||
|
@ -65,27 +71,28 @@ public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
private FixedIndexedWriter<Double> doubleDictionaryWriter;
|
||||
private FrontCodedIntArrayIndexedWriter arrayDictionaryWriter;
|
||||
private FixedIndexedIntWriter arrayElementDictionaryWriter;
|
||||
|
||||
private int rowCount = 0;
|
||||
private boolean closedForWrite = false;
|
||||
|
||||
private boolean dictionarySerialized = false;
|
||||
|
||||
private SingleValueColumnarIntsSerializer encodedValueSerializer;
|
||||
private GenericIndexedWriter<ImmutableBitmap> bitmapIndexWriter;
|
||||
private GenericIndexedWriter<ImmutableBitmap> arrayElementIndexWriter;
|
||||
private MutableBitmap[] bitmaps;
|
||||
private ByteBuffer columnNameBytes = null;
|
||||
private final Int2ObjectRBTreeMap<MutableBitmap> arrayElements = new Int2ObjectRBTreeMap<>();
|
||||
@Nullable
|
||||
private final Byte variantTypeSetByte;
|
||||
|
||||
public VariantArrayColumnSerializer(
|
||||
public VariantColumnSerializer(
|
||||
String name,
|
||||
@Nullable Byte variantTypeSetByte,
|
||||
IndexSpec indexSpec,
|
||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||
Closer closer
|
||||
)
|
||||
{
|
||||
this.name = name;
|
||||
this.variantTypeSetByte = variantTypeSetByte;
|
||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||
this.indexSpec = indexSpec;
|
||||
this.closer = closer;
|
||||
|
@ -254,10 +261,9 @@ public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
}
|
||||
|
||||
ExprEval eval = ExprEval.bestEffortOf(StructuredData.unwrap(selector.getObject()));
|
||||
int[] globalIds = null;
|
||||
if (eval.isArray()) {
|
||||
Object[] array = eval.asArray();
|
||||
globalIds = new int[array.length];
|
||||
int[] globalIds = new int[array.length];
|
||||
for (int i = 0; i < array.length; i++) {
|
||||
if (array[i] == null) {
|
||||
globalIds[i] = 0;
|
||||
|
@ -276,10 +282,35 @@ public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
(id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()
|
||||
).add(rowCount);
|
||||
}
|
||||
final int dictId = dictionaryIdLookup.lookupArray(globalIds);
|
||||
encodedValueSerializer.addValue(dictId);
|
||||
bitmaps[dictId].add(rowCount);
|
||||
} else {
|
||||
final Object o = eval.value();
|
||||
final int dictId;
|
||||
if (o == null) {
|
||||
dictId = 0;
|
||||
} else if (o instanceof String) {
|
||||
dictId = dictionaryIdLookup.lookupString((String) o);
|
||||
} else if (o instanceof Long) {
|
||||
dictId = dictionaryIdLookup.lookupLong((Long) o);
|
||||
} else if (o instanceof Double) {
|
||||
dictId = dictionaryIdLookup.lookupDouble((Double) o);
|
||||
} else {
|
||||
dictId = -1;
|
||||
}
|
||||
Preconditions.checkArgument(dictId >= 0, "unknown global id [%s] for value [%s]", dictId, o);
|
||||
if (dictId != 0) {
|
||||
// treat as single element array
|
||||
arrayElements.computeIfAbsent(
|
||||
dictId,
|
||||
(id) -> indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()
|
||||
).add(rowCount);
|
||||
}
|
||||
encodedValueSerializer.addValue(dictId);
|
||||
bitmaps[dictId].add(rowCount);
|
||||
}
|
||||
final int dictId = globalIds == null ? 0 : dictionaryIdLookup.lookupArray(globalIds);
|
||||
encodedValueSerializer.addValue(dictId);
|
||||
bitmaps[dictId].add(rowCount);
|
||||
|
||||
rowCount++;
|
||||
}
|
||||
|
||||
|
@ -311,6 +342,9 @@ public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
|
||||
long size = 1 + columnNameBytes.capacity();
|
||||
// the value dictionaries, raw column, and null index are all stored in separate files
|
||||
if (variantTypeSetByte != null) {
|
||||
size += 1;
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
|
@ -324,6 +358,9 @@ public class VariantArrayColumnSerializer extends NestedCommonFormatColumnSerial
|
|||
Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?");
|
||||
|
||||
writeV0Header(channel, columnNameBytes);
|
||||
if (variantTypeSetByte != null) {
|
||||
channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte}));
|
||||
}
|
||||
|
||||
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
||||
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
|
@ -28,7 +28,8 @@ import java.io.IOException;
|
|||
import java.nio.channels.WritableByteChannel;
|
||||
|
||||
/**
|
||||
* Literal field writer for mixed type nested columns of {@link NestedDataColumnSerializerV4}
|
||||
* Nested field writer for mixed type scalar or array columns of {@link NestedDataColumnSerializerV4} and
|
||||
* {@link NestedDataColumnSerializer}.
|
||||
*/
|
||||
public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedFieldColumnWriter<Object>
|
||||
{
|
||||
|
@ -47,6 +48,7 @@ public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedField
|
|||
@Override
|
||||
Object processValue(int row, Object value)
|
||||
{
|
||||
// replace arrays represented as Object[] with int[] composed of the global ids
|
||||
if (value instanceof Object[]) {
|
||||
Object[] array = (Object[]) value;
|
||||
final int[] globalIds = new int[array.length];
|
||||
|
@ -66,6 +68,7 @@ public final class VariantFieldColumnWriter extends GlobalDictionaryEncodedField
|
|||
}
|
||||
return globalIds;
|
||||
}
|
||||
// non-arrays can be passed directly through
|
||||
return super.processValue(row, value);
|
||||
}
|
||||
|
||||
|
|
|
@ -33,13 +33,25 @@ import org.apache.druid.segment.nested.NestedDataColumnSupplier;
|
|||
import org.apache.druid.segment.nested.ScalarDoubleColumnAndIndexSupplier;
|
||||
import org.apache.druid.segment.nested.ScalarLongColumnAndIndexSupplier;
|
||||
import org.apache.druid.segment.nested.ScalarStringColumnAndIndexSupplier;
|
||||
import org.apache.druid.segment.nested.VariantArrayColumnAndIndexSupplier;
|
||||
import org.apache.druid.segment.nested.VariantColumnAndIndexSupplier;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
/**
|
||||
* {@link ColumnPartSerde} shared by all {@link NestedCommonFormatColumn}. The {@link #logicalType} defines the native
|
||||
* Druid type of the column to use for things like {@link org.apache.druid.segment.column.ColumnCapabilities} and
|
||||
* {@link #hasNulls} if any null values are present. If {@link #isVariantType} is set, the column internally is
|
||||
* composed of a mix of types, this is currently only used by {@link VariantColumnAndIndexSupplier}.
|
||||
*
|
||||
* @see ScalarDoubleColumnAndIndexSupplier
|
||||
* @see ScalarLongColumnAndIndexSupplier
|
||||
* @see ScalarStringColumnAndIndexSupplier
|
||||
* @see VariantColumnAndIndexSupplier
|
||||
* @see NestedDataColumnSupplier
|
||||
*/
|
||||
public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
||||
{
|
||||
public static SerializerBuilder serializerBuilder()
|
||||
|
@ -62,15 +74,17 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
public static NestedCommonFormatColumnPartSerde createDeserializer(
|
||||
@JsonProperty("logicalType") ColumnType logicalType,
|
||||
@JsonProperty("hasNulls") boolean hasNulls,
|
||||
@JsonProperty("isVariantType") boolean isVariantType,
|
||||
@JsonProperty("byteOrder") ByteOrder byteOrder,
|
||||
@JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory
|
||||
)
|
||||
{
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, byteOrder, bitmapSerdeFactory, null);
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, isVariantType, byteOrder, bitmapSerdeFactory, null);
|
||||
}
|
||||
|
||||
private final ColumnType logicalType;
|
||||
private final boolean hasNulls;
|
||||
private final boolean isVariantType;
|
||||
private final ByteOrder byteOrder;
|
||||
private final BitmapSerdeFactory bitmapSerdeFactory;
|
||||
|
||||
|
@ -81,6 +95,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
private NestedCommonFormatColumnPartSerde(
|
||||
ColumnType logicalType,
|
||||
boolean hasNulls,
|
||||
boolean isVariant,
|
||||
ByteOrder byteOrder,
|
||||
BitmapSerdeFactory bitmapSerdeFactory,
|
||||
@Nullable Serializer serializer
|
||||
|
@ -88,6 +103,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
{
|
||||
this.logicalType = logicalType;
|
||||
this.hasNulls = hasNulls;
|
||||
this.isVariantType = isVariant;
|
||||
this.byteOrder = byteOrder;
|
||||
this.bitmapSerdeFactory = bitmapSerdeFactory;
|
||||
this.serializer = serializer;
|
||||
|
@ -104,6 +120,26 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
@Override
|
||||
public Deserializer getDeserializer()
|
||||
{
|
||||
if (isVariantType || logicalType.isArray()) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
|
||||
logicalType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
}
|
||||
if (logicalType.is(ValueType.STRING)) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read(
|
||||
|
@ -118,7 +154,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setStandardTypeColumnSupplier(supplier);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
|
@ -138,7 +174,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setStandardTypeColumnSupplier(supplier);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
|
@ -158,32 +194,12 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setStandardTypeColumnSupplier(supplier);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setIndexSupplier(supplier, true, false);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
}
|
||||
if (logicalType.isArray()) {
|
||||
return ((buffer, builder, columnConfig) -> {
|
||||
VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read(
|
||||
logicalType,
|
||||
byteOrder,
|
||||
bitmapSerdeFactory,
|
||||
buffer,
|
||||
builder,
|
||||
columnConfig
|
||||
);
|
||||
ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder();
|
||||
capabilitiesBuilder.setDictionaryEncoded(true);
|
||||
capabilitiesBuilder.setDictionaryValuesSorted(true);
|
||||
capabilitiesBuilder.setDictionaryValuesUnique(true);
|
||||
builder.setType(logicalType);
|
||||
builder.setStandardTypeColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue()));
|
||||
builder.setFilterable(true);
|
||||
});
|
||||
}
|
||||
return (buffer, builder, columnConfig) -> {
|
||||
NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read(
|
||||
hasNulls,
|
||||
|
@ -200,7 +216,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
ColumnType simpleType = supplier.getLogicalType();
|
||||
ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType;
|
||||
builder.setType(logicalType);
|
||||
builder.setStandardTypeColumnSupplier(supplier);
|
||||
builder.setNestedCommonFormatColumnSupplier(supplier);
|
||||
builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls));
|
||||
builder.setFilterable(true);
|
||||
};
|
||||
|
@ -218,6 +234,12 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
return hasNulls;
|
||||
}
|
||||
|
||||
@JsonProperty("isVariantType")
|
||||
public boolean isVariantType()
|
||||
{
|
||||
return isVariantType;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public ByteOrder getByteOrder()
|
||||
{
|
||||
|
@ -234,6 +256,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
{
|
||||
private ColumnType logicalType;
|
||||
private boolean hasNulls;
|
||||
private boolean isVariantType;
|
||||
private ByteOrder byteOrder = ByteOrder.nativeOrder();
|
||||
BitmapSerdeFactory bitmapSerdeFactory;
|
||||
@Nullable
|
||||
|
@ -245,6 +268,12 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
return this;
|
||||
}
|
||||
|
||||
public SerializerBuilder isVariantType(boolean isVariant)
|
||||
{
|
||||
this.isVariantType = isVariant;
|
||||
return this;
|
||||
}
|
||||
|
||||
public SerializerBuilder withSerializer(final Serializer serializer)
|
||||
{
|
||||
this.serializer = serializer;
|
||||
|
@ -271,7 +300,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde
|
|||
|
||||
public NestedCommonFormatColumnPartSerde build()
|
||||
{
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, byteOrder, bitmapSerdeFactory, serializer);
|
||||
return new NestedCommonFormatColumnPartSerde(logicalType, hasNulls, isVariantType, byteOrder, bitmapSerdeFactory, serializer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -26,10 +26,11 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.common.guava.GuavaUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Numbers;
|
||||
import org.apache.druid.math.expr.Evals;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.query.dimension.DimensionSpec;
|
||||
import org.apache.druid.query.extraction.ExtractionFn;
|
||||
|
@ -51,11 +52,13 @@ import org.apache.druid.segment.column.ColumnHolder;
|
|||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.NumericColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.column.ValueTypes;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.data.ReadableOffset;
|
||||
import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn;
|
||||
import org.apache.druid.segment.nested.NestedCommonFormatColumn;
|
||||
import org.apache.druid.segment.nested.NestedDataComplexColumn;
|
||||
import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
|
||||
import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn;
|
||||
|
@ -63,8 +66,10 @@ import org.apache.druid.segment.nested.NestedPathArrayElement;
|
|||
import org.apache.druid.segment.nested.NestedPathFinder;
|
||||
import org.apache.druid.segment.nested.NestedPathPart;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.segment.nested.VariantArrayColumn;
|
||||
import org.apache.druid.segment.nested.VariantColumn;
|
||||
import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier;
|
||||
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
|
||||
import org.apache.druid.segment.vector.BaseFloatVectorValueSelector;
|
||||
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
|
||||
import org.apache.druid.segment.vector.NilVectorSelector;
|
||||
import org.apache.druid.segment.vector.ReadableVectorInspector;
|
||||
|
@ -78,6 +83,7 @@ import java.nio.ByteBuffer;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Optimized virtual column that can make direct selectors into a {@link NestedDataComplexColumn} or any associated
|
||||
|
@ -294,8 +300,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
);
|
||||
}
|
||||
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) {
|
||||
final VariantArrayColumn<?> arrayColumn = (VariantArrayColumn<?>) theColumn;
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
|
||||
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
|
||||
ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset);
|
||||
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
|
@ -365,7 +371,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
if (parts.isEmpty()) {
|
||||
// dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat)
|
||||
// so we want to wrap their selector in a "best effort" casting selector to implement them
|
||||
if (theColumn instanceof DictionaryEncodedColumn) {
|
||||
if (theColumn instanceof DictionaryEncodedColumn && !(theColumn instanceof VariantColumn)) {
|
||||
final DictionaryEncodedColumn<?> column = (DictionaryEncodedColumn<?>) theColumn;
|
||||
return new BestEffortCastingValueSelector(column.makeDimensionSelector(offset, null));
|
||||
}
|
||||
|
@ -374,8 +380,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
return theColumn.makeColumnValueSelector(offset);
|
||||
}
|
||||
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) {
|
||||
final VariantArrayColumn<?> arrayColumn = (VariantArrayColumn<?>) theColumn;
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
|
||||
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
|
||||
ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset);
|
||||
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
|
@ -532,8 +538,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
return column.makeVectorObjectSelector(offset);
|
||||
}
|
||||
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantArrayColumn) {
|
||||
final VariantArrayColumn<?> arrayColumn = (VariantArrayColumn<?>) column;
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantColumn) {
|
||||
final VariantColumn<?> arrayColumn = (VariantColumn<?>) column;
|
||||
VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset);
|
||||
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
|
@ -601,131 +607,360 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
if (!(theColumn instanceof NestedDataComplexColumn)) {
|
||||
|
||||
if (parts.isEmpty()) {
|
||||
ColumnCapabilities capabilities = holder.getCapabilities();
|
||||
if (theColumn instanceof DictionaryEncodedColumn) {
|
||||
return ExpressionVectorSelectors.castObjectSelectorToNumeric(
|
||||
offset,
|
||||
this.columnName,
|
||||
theColumn.makeVectorObjectSelector(offset),
|
||||
capabilities.toColumnType(),
|
||||
expectedType
|
||||
);
|
||||
final VectorObjectSelector delegate = theColumn.makeVectorObjectSelector(offset);
|
||||
if (expectedType != null && expectedType.is(ValueType.LONG)) {
|
||||
return new BaseLongVectorValueSelector(offset)
|
||||
{
|
||||
private int currentOffsetId = ReadableVectorInspector.NULL_ID;
|
||||
private final long[] longs = new long[delegate.getMaxVectorSize()];
|
||||
@Nullable
|
||||
private boolean[] nulls = null;
|
||||
|
||||
@Override
|
||||
public long[] getLongVector()
|
||||
{
|
||||
computeLongs();
|
||||
return longs;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
computeLongs();
|
||||
return nulls;
|
||||
}
|
||||
|
||||
private void computeLongs()
|
||||
{
|
||||
if (currentOffsetId != offset.getId()) {
|
||||
currentOffsetId = offset.getId();
|
||||
final Object[] values = delegate.getObjectVector();
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Number n = ExprEval.computeNumber(Evals.asString(values[i]));
|
||||
if (n != null) {
|
||||
longs[i] = n.longValue();
|
||||
if (nulls != null) {
|
||||
nulls[i] = false;
|
||||
}
|
||||
} else {
|
||||
if (nulls == null) {
|
||||
nulls = new boolean[offset.getMaxVectorSize()];
|
||||
}
|
||||
nulls[i] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
} else if (expectedType != null && expectedType.is(ValueType.FLOAT)) {
|
||||
return new BaseFloatVectorValueSelector(offset)
|
||||
{
|
||||
private int currentOffsetId = ReadableVectorInspector.NULL_ID;
|
||||
private final float[] floats = new float[delegate.getMaxVectorSize()];
|
||||
@Nullable
|
||||
private boolean[] nulls = null;
|
||||
|
||||
@Override
|
||||
public float[] getFloatVector()
|
||||
{
|
||||
computeFloats();
|
||||
return floats;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
computeFloats();
|
||||
return nulls;
|
||||
}
|
||||
|
||||
private void computeFloats()
|
||||
{
|
||||
if (currentOffsetId != offset.getId()) {
|
||||
currentOffsetId = offset.getId();
|
||||
final Object[] values = delegate.getObjectVector();
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Number n = ExprEval.computeNumber(Evals.asString(values[i]));
|
||||
if (n != null) {
|
||||
floats[i] = n.floatValue();
|
||||
if (nulls != null) {
|
||||
nulls[i] = false;
|
||||
}
|
||||
} else {
|
||||
if (nulls == null) {
|
||||
nulls = new boolean[offset.getMaxVectorSize()];
|
||||
}
|
||||
nulls[i] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new BaseDoubleVectorValueSelector(offset)
|
||||
{
|
||||
private int currentOffsetId = ReadableVectorInspector.NULL_ID;
|
||||
private final double[] doubles = new double[delegate.getMaxVectorSize()];
|
||||
@Nullable
|
||||
private boolean[] nulls = null;
|
||||
@Override
|
||||
public double[] getDoubleVector()
|
||||
{
|
||||
computeDoubles();
|
||||
return doubles;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
computeDoubles();
|
||||
return nulls;
|
||||
}
|
||||
|
||||
private void computeDoubles()
|
||||
{
|
||||
if (currentOffsetId != offset.getId()) {
|
||||
currentOffsetId = offset.getId();
|
||||
final Object[] values = delegate.getObjectVector();
|
||||
for (int i = 0; i < values.length; i++) {
|
||||
Number n = ExprEval.computeNumber(Evals.asString(values[i]));
|
||||
if (n != null) {
|
||||
doubles[i] = n.doubleValue();
|
||||
if (nulls != null) {
|
||||
nulls[i] = false;
|
||||
}
|
||||
} else {
|
||||
if (nulls == null) {
|
||||
nulls = new boolean[offset.getMaxVectorSize()];
|
||||
}
|
||||
nulls[i] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
return theColumn.makeVectorValueSelector(offset);
|
||||
}
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantArrayColumn) {
|
||||
final VariantArrayColumn<?> arrayColumn = (VariantArrayColumn<?>) theColumn;
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
|
||||
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
|
||||
VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset);
|
||||
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex();
|
||||
if (elementNumber < 0) {
|
||||
throw new IAE("Cannot make array element selector, negative array index not supported");
|
||||
}
|
||||
|
||||
return new VectorValueSelector()
|
||||
{
|
||||
private final long[] longs = new long[offset.getMaxVectorSize()];
|
||||
private final double[] doubles = new double[offset.getMaxVectorSize()];
|
||||
private final float[] floats = new float[offset.getMaxVectorSize()];
|
||||
private final boolean[] nulls = new boolean[offset.getMaxVectorSize()];
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
private void computeNumbers()
|
||||
if (expectedType != null && expectedType.is(ValueType.LONG)) {
|
||||
return new BaseLongVectorValueSelector(offset)
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
final Object[] maybeArrays = arraySelector.getObjectVector();
|
||||
for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
|
||||
Object maybeArray = maybeArrays[i];
|
||||
if (maybeArray instanceof Object[]) {
|
||||
Object[] anArray = (Object[]) maybeArray;
|
||||
if (elementNumber < anArray.length) {
|
||||
if (anArray[elementNumber] instanceof Number) {
|
||||
Number n = (Number) anArray[elementNumber];
|
||||
longs[i] = n.longValue();
|
||||
doubles[i] = n.doubleValue();
|
||||
floats[i] = n.floatValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
Double d = anArray[elementNumber] instanceof String
|
||||
? Doubles.tryParse((String) anArray[elementNumber])
|
||||
: null;
|
||||
if (d != null) {
|
||||
longs[i] = d.longValue();
|
||||
doubles[i] = d;
|
||||
floats[i] = d.floatValue();
|
||||
private final long[] longs = new long[offset.getMaxVectorSize()];
|
||||
private final boolean[] nulls = new boolean[offset.getMaxVectorSize()];
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
private void computeNumbers()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
final Object[] maybeArrays = arraySelector.getObjectVector();
|
||||
for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
|
||||
Object maybeArray = maybeArrays[i];
|
||||
if (maybeArray instanceof Object[]) {
|
||||
Object[] anArray = (Object[]) maybeArray;
|
||||
if (elementNumber < anArray.length) {
|
||||
if (anArray[elementNumber] instanceof Number) {
|
||||
Number n = (Number) anArray[elementNumber];
|
||||
longs[i] = n.longValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
nullElement(i);
|
||||
Double d = anArray[elementNumber] instanceof String
|
||||
? Doubles.tryParse((String) anArray[elementNumber])
|
||||
: null;
|
||||
if (d != null) {
|
||||
longs[i] = d.longValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
longs[i] = 0L;
|
||||
nulls[i] = true;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
nullElement(i);
|
||||
}
|
||||
} else {
|
||||
// not an array?
|
||||
nullElement(i);
|
||||
}
|
||||
} else {
|
||||
// not an array?
|
||||
nullElement(i);
|
||||
}
|
||||
id = offset.getId();
|
||||
}
|
||||
id = offset.getId();
|
||||
}
|
||||
}
|
||||
|
||||
private void nullElement(int i)
|
||||
{
|
||||
longs[i] = 0L;
|
||||
doubles[i] = 0L;
|
||||
floats[i] = 0L;
|
||||
nulls[i] = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long[] getLongVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
private void nullElement(int i)
|
||||
{
|
||||
longs[i] = 0L;
|
||||
nulls[i] = true;
|
||||
}
|
||||
return longs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float[] getFloatVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
@Override
|
||||
public long[] getLongVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return longs;
|
||||
}
|
||||
return floats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getDoubleVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return nulls;
|
||||
}
|
||||
return doubles;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
};
|
||||
} else if (expectedType != null && expectedType.is(ValueType.FLOAT)) {
|
||||
return new BaseFloatVectorValueSelector(offset)
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
private final float[] floats = new float[offset.getMaxVectorSize()];
|
||||
private final boolean[] nulls = new boolean[offset.getMaxVectorSize()];
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
private void computeNumbers()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
final Object[] maybeArrays = arraySelector.getObjectVector();
|
||||
for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
|
||||
Object maybeArray = maybeArrays[i];
|
||||
if (maybeArray instanceof Object[]) {
|
||||
Object[] anArray = (Object[]) maybeArray;
|
||||
if (elementNumber < anArray.length) {
|
||||
if (anArray[elementNumber] instanceof Number) {
|
||||
Number n = (Number) anArray[elementNumber];
|
||||
floats[i] = n.floatValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
Double d = anArray[elementNumber] instanceof String
|
||||
? Doubles.tryParse((String) anArray[elementNumber])
|
||||
: null;
|
||||
if (d != null) {
|
||||
floats[i] = d.floatValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
nullElement(i);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
nullElement(i);
|
||||
}
|
||||
} else {
|
||||
// not an array?
|
||||
nullElement(i);
|
||||
}
|
||||
}
|
||||
id = offset.getId();
|
||||
}
|
||||
}
|
||||
return nulls;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxVectorSize()
|
||||
{
|
||||
return arraySelector.getMaxVectorSize();
|
||||
}
|
||||
private void nullElement(int i)
|
||||
{
|
||||
floats[i] = 0f;
|
||||
nulls[i] = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getCurrentVectorSize()
|
||||
@Override
|
||||
public float[] getFloatVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return floats;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return nulls;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new BaseDoubleVectorValueSelector(offset)
|
||||
{
|
||||
return arraySelector.getCurrentVectorSize();
|
||||
}
|
||||
};
|
||||
private final double[] doubles = new double[offset.getMaxVectorSize()];
|
||||
private final boolean[] nulls = new boolean[offset.getMaxVectorSize()];
|
||||
private int id = ReadableVectorInspector.NULL_ID;
|
||||
|
||||
private void computeNumbers()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
final Object[] maybeArrays = arraySelector.getObjectVector();
|
||||
for (int i = 0; i < arraySelector.getCurrentVectorSize(); i++) {
|
||||
Object maybeArray = maybeArrays[i];
|
||||
if (maybeArray instanceof Object[]) {
|
||||
Object[] anArray = (Object[]) maybeArray;
|
||||
if (elementNumber < anArray.length) {
|
||||
if (anArray[elementNumber] instanceof Number) {
|
||||
Number n = (Number) anArray[elementNumber];
|
||||
doubles[i] = n.doubleValue();
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
Double d = anArray[elementNumber] instanceof String
|
||||
? Doubles.tryParse((String) anArray[elementNumber])
|
||||
: null;
|
||||
if (d != null) {
|
||||
doubles[i] = d;
|
||||
nulls[i] = false;
|
||||
} else {
|
||||
nullElement(i);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
nullElement(i);
|
||||
}
|
||||
} else {
|
||||
// not an array?
|
||||
nullElement(i);
|
||||
}
|
||||
}
|
||||
id = offset.getId();
|
||||
}
|
||||
}
|
||||
|
||||
private void nullElement(int i)
|
||||
{
|
||||
doubles[i] = 0.0;
|
||||
nulls[i] = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double[] getDoubleVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return doubles;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public boolean[] getNullVector()
|
||||
{
|
||||
if (offset.getId() != id) {
|
||||
computeNumbers();
|
||||
}
|
||||
return nulls;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
return NilVectorSelector.create(offset);
|
||||
}
|
||||
|
@ -776,7 +1011,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
nullVector = new boolean[objectSelector.getMaxVectorSize()];
|
||||
}
|
||||
longVector[i] = 0L;
|
||||
nullVector[i] = NullHandling.sqlCompatible();
|
||||
nullVector[i] = true;
|
||||
} else {
|
||||
Long l;
|
||||
if (v instanceof Number) {
|
||||
|
@ -794,7 +1029,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
nullVector = new boolean[objectSelector.getMaxVectorSize()];
|
||||
}
|
||||
longVector[i] = 0L;
|
||||
nullVector[i] = NullHandling.sqlCompatible();
|
||||
nullVector[i] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -839,7 +1074,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
nullVector = new boolean[objectSelector.getMaxVectorSize()];
|
||||
}
|
||||
doubleVector[i] = 0.0;
|
||||
nullVector[i] = NullHandling.sqlCompatible();
|
||||
nullVector[i] = true;
|
||||
} else {
|
||||
Double d;
|
||||
if (v instanceof Number) {
|
||||
|
@ -857,7 +1092,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
nullVector = new boolean[objectSelector.getMaxVectorSize()];
|
||||
}
|
||||
doubleVector[i] = 0.0;
|
||||
nullVector[i] = NullHandling.sqlCompatible();
|
||||
nullVector[i] = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -879,13 +1114,41 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
}
|
||||
BaseColumn theColumn = holder.getColumn();
|
||||
if (theColumn instanceof CompressedNestedDataComplexColumn) {
|
||||
return ((CompressedNestedDataComplexColumn<?>) theColumn).getColumnIndexSupplier(parts);
|
||||
final CompressedNestedDataComplexColumn<?> nestedColumn = (CompressedNestedDataComplexColumn<?>) theColumn;
|
||||
final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(parts);
|
||||
if (expectedType != null) {
|
||||
final Set<ColumnType> types = nestedColumn.getColumnTypes(parts);
|
||||
// if the expected output type is numeric but not all of the input types are numeric, we might have additional
|
||||
// null values than what the null value bitmap is tracking, wrap it
|
||||
if (expectedType.isNumeric() && types.stream().anyMatch(t -> !t.isNumeric())) {
|
||||
return NoIndexesColumnIndexSupplier.getInstance();
|
||||
}
|
||||
}
|
||||
return nestedColumnPathIndexSupplier;
|
||||
}
|
||||
if (parts.isEmpty()) {
|
||||
return holder.getIndexSupplier();
|
||||
final ColumnIndexSupplier baseIndexSupplier = holder.getIndexSupplier();
|
||||
if (expectedType != null) {
|
||||
if (theColumn instanceof NumericColumn) {
|
||||
return baseIndexSupplier;
|
||||
}
|
||||
if (theColumn instanceof NestedCommonFormatColumn) {
|
||||
final NestedCommonFormatColumn commonFormat = (NestedCommonFormatColumn) theColumn;
|
||||
if (expectedType.isNumeric() && !commonFormat.getLogicalType().isNumeric()) {
|
||||
return NoIndexesColumnIndexSupplier.getInstance();
|
||||
}
|
||||
} else {
|
||||
return expectedType.isNumeric() ? NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier;
|
||||
}
|
||||
}
|
||||
return baseIndexSupplier;
|
||||
}
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && !hasNegativeArrayIndex && theColumn instanceof VariantArrayColumn) {
|
||||
return holder.getIndexSupplier();
|
||||
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
|
||||
// cannot use the array column index supplier directly, in the future array columns should expose a function
|
||||
// with a signature like 'getArrayElementIndexSupplier(int index)' to allow getting indexes for specific elements
|
||||
// if we want to support this stuff. Right now VariantArrayColumn doesn't actually retain enough information about
|
||||
// what positions the values are in to support doing anything cool here, so we just return 'no indexes'
|
||||
return NoIndexesColumnIndexSupplier.getInstance();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -904,7 +1167,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
// from here
|
||||
return ColumnCapabilitiesImpl.createDefault()
|
||||
.setType(expectedType != null ? expectedType : ColumnType.STRING)
|
||||
.setHasNulls(expectedType == null || !expectedType.isNumeric() || (expectedType.isNumeric() && NullHandling.sqlCompatible()));
|
||||
.setHasNulls(true);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -934,8 +1197,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
.setDictionaryValuesSorted(useDictionary)
|
||||
.setDictionaryValuesUnique(useDictionary)
|
||||
.setHasBitmapIndexes(useDictionary)
|
||||
.setHasNulls(expectedType == null || (expectedType.isNumeric()
|
||||
&& NullHandling.sqlCompatible()));
|
||||
.setHasNulls(true);
|
||||
}
|
||||
// column is not nested, use underlying column capabilities, adjusted for expectedType as necessary
|
||||
if (parts.isEmpty()) {
|
||||
|
@ -1011,7 +1273,10 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
*/
|
||||
public static class RawFieldLiteralColumnValueSelector extends RawFieldColumnSelector
|
||||
{
|
||||
public RawFieldLiteralColumnValueSelector(ColumnValueSelector baseSelector, List<NestedPathPart> parts)
|
||||
public RawFieldLiteralColumnValueSelector(
|
||||
ColumnValueSelector baseSelector,
|
||||
List<NestedPathPart> parts
|
||||
)
|
||||
{
|
||||
super(baseSelector, parts);
|
||||
}
|
||||
|
@ -1337,7 +1602,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
|
|||
if (row.size() != 1) {
|
||||
return true;
|
||||
}
|
||||
return Doubles.tryParse(lookupName(row.get(0))) == null;
|
||||
final String s = lookupName(row.get(0));
|
||||
return s == null || Doubles.tryParse(s) == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.IAE;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.column.TypeStrategies;
|
||||
import org.apache.druid.segment.column.TypeStrategiesTest;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -272,7 +273,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.STRING)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<STRING> to STRING", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<STRING>] to [STRING]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -282,7 +283,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.LONG)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<STRING> to LONG", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<STRING>] to [LONG]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -292,7 +293,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.DOUBLE)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<STRING> to DOUBLE", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<STRING>] to [DOUBLE]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -302,7 +303,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.STRING)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<LONG> to STRING", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<LONG>] to [STRING]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -312,7 +313,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.LONG)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<LONG> to LONG", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<LONG>] to [LONG]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -322,7 +323,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.DOUBLE)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<LONG> to DOUBLE", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<LONG>] to [DOUBLE]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -332,7 +333,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.STRING)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<DOUBLE> to STRING", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<DOUBLE>] to [STRING]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -342,7 +343,7 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.LONG)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<DOUBLE> to LONG", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<DOUBLE>] to [LONG]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -352,7 +353,285 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
IAE.class,
|
||||
() -> ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.DOUBLE)
|
||||
);
|
||||
Assert.assertEquals("invalid type cannot cast ARRAY<DOUBLE> to DOUBLE", t.getMessage());
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<DOUBLE>] to [DOUBLE]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNestedDataCast()
|
||||
{
|
||||
ExprEval cast;
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, "hello").castTo(ExpressionType.STRING);
|
||||
Assert.assertEquals("hello", cast.value());
|
||||
Assert.assertEquals("hello", ExprEval.ofComplex(ExpressionType.NESTED_DATA, "hello").asString());
|
||||
Assert.assertEquals(ExpressionType.STRING, cast.type());
|
||||
|
||||
cast = ExprEval.of("hello").castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertEquals("hello", cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, 123L).castTo(ExpressionType.STRING);
|
||||
Assert.assertEquals("123", cast.value());
|
||||
Assert.assertEquals(ExpressionType.STRING, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, 123L).castTo(ExpressionType.LONG);
|
||||
Assert.assertEquals(123L, cast.value());
|
||||
Assert.assertEquals(123L, ExprEval.ofComplex(ExpressionType.NESTED_DATA, 123L).asLong());
|
||||
Assert.assertEquals(ExpressionType.LONG, cast.type());
|
||||
|
||||
cast = ExprEval.of(123L).castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertEquals(123L, cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, 123L).castTo(ExpressionType.DOUBLE);
|
||||
Assert.assertEquals(123.0, cast.value());
|
||||
Assert.assertEquals(123.0, ExprEval.ofComplex(ExpressionType.NESTED_DATA, 123L).asDouble(), 0.0);
|
||||
Assert.assertEquals(ExpressionType.DOUBLE, cast.type());
|
||||
|
||||
cast = ExprEval.of(12.3).castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertEquals(12.3, cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableList.of("a", "b", "c")).castTo(ExpressionType.STRING_ARRAY);
|
||||
Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) cast.value());
|
||||
Assert.assertArrayEquals(
|
||||
new Object[]{"a", "b", "c"},
|
||||
ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableList.of("a", "b", "c")).asArray()
|
||||
);
|
||||
Assert.assertEquals(ExpressionType.STRING_ARRAY, cast.type());
|
||||
|
||||
cast = ExprEval.ofArray(ExpressionType.STRING_ARRAY, new Object[]{"a", "b", "c"}).castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableList.of(1L, 2L, 3L)).castTo(ExpressionType.LONG_ARRAY);
|
||||
Assert.assertArrayEquals(new Object[]{1L, 2L, 3L}, (Object[]) cast.value());
|
||||
Assert.assertArrayEquals(
|
||||
new Object[]{1L, 2L, 3L},
|
||||
ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableList.of(1L, 2L, 3L)).asArray()
|
||||
);
|
||||
Assert.assertEquals(ExpressionType.LONG_ARRAY, cast.type());
|
||||
|
||||
cast = ExprEval.ofArray(ExpressionType.LONG_ARRAY, new Object[]{1L, 2L, 3L}).castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertArrayEquals(new Object[]{1L, 2L, 3L}, (Object[]) cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
cast = ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableList.of(1L, 2L, 3L)).castTo(ExpressionType.DOUBLE_ARRAY);
|
||||
Assert.assertArrayEquals(new Object[]{1.0, 2.0, 3.0}, (Object[]) cast.value());
|
||||
Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, cast.type());
|
||||
|
||||
cast = ExprEval.ofArray(ExpressionType.DOUBLE_ARRAY, new Object[]{1.1, 2.2, 3.3}).castTo(ExpressionType.NESTED_DATA);
|
||||
Assert.assertArrayEquals(new Object[]{1.1, 2.2, 3.3}, (Object[]) cast.value());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
|
||||
|
||||
ExpressionType nestedArray = ExpressionTypeFactory.getInstance().ofArray(ExpressionType.NESTED_DATA);
|
||||
cast = ExprEval.ofComplex(
|
||||
ExpressionType.NESTED_DATA,
|
||||
ImmutableList.of(ImmutableMap.of("x", 1, "y", 2), ImmutableMap.of("x", 3, "y", 4))
|
||||
).castTo(nestedArray);
|
||||
Assert.assertArrayEquals(
|
||||
new Object[]{
|
||||
ImmutableMap.of("x", 1, "y", 2),
|
||||
ImmutableMap.of("x", 3, "y", 4)
|
||||
},
|
||||
(Object[]) cast.value()
|
||||
);
|
||||
Assert.assertEquals(nestedArray, cast.type());
|
||||
Assert.assertArrayEquals(
|
||||
new Object[]{
|
||||
ImmutableMap.of("x", 1, "y", 2),
|
||||
ImmutableMap.of("x", 3, "y", 4)
|
||||
},
|
||||
ExprEval.ofComplex(
|
||||
ExpressionType.NESTED_DATA,
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("x", 1, "y", 2),
|
||||
ImmutableMap.of("x", 3, "y", 4)
|
||||
)
|
||||
).asArray()
|
||||
);
|
||||
|
||||
Assert.assertThrows(IAE.class, () -> ExprEval.ofLong(1234L).castTo(nestedArray));
|
||||
Assert.assertThrows(IAE.class, () -> ExprEval.of("hello").castTo(nestedArray));
|
||||
Assert.assertThrows(IAE.class, () -> ExprEval.ofDouble(1.234).castTo(nestedArray));
|
||||
Assert.assertThrows(IAE.class, () -> ExprEval.ofComplex(ExpressionType.NESTED_DATA, 1234L).castTo(nestedArray));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNestedAsOtherStuff()
|
||||
{
|
||||
ExprEval eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, StructuredData.wrap(true));
|
||||
Assert.assertTrue(eval.asBoolean());
|
||||
Assert.assertFalse(eval.isNumericNull());
|
||||
Assert.assertEquals(1, eval.asInt());
|
||||
Assert.assertEquals(1L, eval.asLong());
|
||||
Assert.assertEquals(1.0, eval.asDouble(), 0.0);
|
||||
|
||||
Assert.assertTrue(ExprEval.ofComplex(ExpressionType.NESTED_DATA, true).asBoolean());
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, false);
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertFalse(eval.isNumericNull());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0L, eval.asLong());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, "true");
|
||||
Assert.assertTrue(eval.asBoolean());
|
||||
Assert.assertFalse(eval.isNumericNull());
|
||||
Assert.assertEquals(1L, eval.asLong());
|
||||
Assert.assertEquals(1, eval.asInt());
|
||||
Assert.assertEquals(1.0, eval.asDouble(), 0.0);
|
||||
|
||||
Assert.assertTrue(ExprEval.ofComplex(ExpressionType.NESTED_DATA, StructuredData.wrap("true")).asBoolean());
|
||||
Assert.assertTrue(ExprEval.ofComplex(ExpressionType.NESTED_DATA, "TRUE").asBoolean());
|
||||
Assert.assertTrue(ExprEval.ofComplex(ExpressionType.NESTED_DATA, "True").asBoolean());
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, StructuredData.wrap(1L));
|
||||
Assert.assertTrue(eval.asBoolean());
|
||||
Assert.assertFalse(eval.isNumericNull());
|
||||
Assert.assertEquals(1L, eval.asLong());
|
||||
Assert.assertEquals(1, eval.asInt());
|
||||
Assert.assertEquals(1.0, eval.asDouble(), 0.0);
|
||||
|
||||
Assert.assertTrue(ExprEval.ofComplex(ExpressionType.NESTED_DATA, 1L).asBoolean());
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, StructuredData.wrap(1.23));
|
||||
Assert.assertTrue(eval.asBoolean());
|
||||
Assert.assertFalse(eval.isNumericNull());
|
||||
Assert.assertEquals(1L, eval.asLong());
|
||||
Assert.assertEquals(1, eval.asInt());
|
||||
Assert.assertEquals(1.23, eval.asDouble(), 0.0);
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, "hello");
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertTrue(eval.isNumericNull());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0L, eval.asLong());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, Arrays.asList("1", "2", "3"));
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertTrue(eval.isNumericNull());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0L, eval.asLong());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
Assert.assertArrayEquals(new Object[]{"1", "2", "3"}, eval.asArray());
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, Arrays.asList(1L, 2L, 3L));
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertTrue(eval.isNumericNull());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0L, eval.asLong());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
Assert.assertArrayEquals(new Object[]{1L, 2L, 3L}, eval.asArray());
|
||||
|
||||
eval = ExprEval.ofComplex(ExpressionType.NESTED_DATA, Arrays.asList(1.1, 2.2, 3.3));
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertTrue(eval.isNumericNull());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0L, eval.asLong());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
Assert.assertArrayEquals(new Object[]{1.1, 2.2, 3.3}, eval.asArray());
|
||||
|
||||
eval = ExprEval.ofComplex(
|
||||
ExpressionType.NESTED_DATA,
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("x", 1, "y", 2),
|
||||
ImmutableMap.of("x", 3, "y", 4)
|
||||
)
|
||||
);
|
||||
Assert.assertFalse(eval.asBoolean());
|
||||
Assert.assertEquals(0, eval.asLong());
|
||||
Assert.assertEquals(0, eval.asInt());
|
||||
Assert.assertEquals(0.0, eval.asDouble(), 0.0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonNestedComplexCastThrows()
|
||||
{
|
||||
ExpressionType someComplex = ExpressionTypeFactory.getInstance().ofComplex("tester");
|
||||
Throwable t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.STRING)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [STRING]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.LONG)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [LONG]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.DOUBLE)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [DOUBLE]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.STRING_ARRAY)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [ARRAY<STRING>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.LONG_ARRAY)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [ARRAY<LONG>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.DOUBLE_ARRAY)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [ARRAY<DOUBLE>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofType(someComplex, "hello").castTo(ExpressionType.NESTED_DATA)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<tester>] to [COMPLEX<json>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.of("hello").castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [STRING] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.of(123L).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [LONG] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.of(1.23).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [DOUBLE] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofStringArray(new Object[]{"a", "b", "c"}).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<STRING>] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofLongArray(new Object[]{1L, 2L, 3L}).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<LONG>] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofDoubleArray(new Object[]{1.1, 2.2, 3.3}).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [ARRAY<DOUBLE>] to [COMPLEX<tester>]", t.getMessage());
|
||||
|
||||
t = Assert.assertThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> ExprEval.ofComplex(ExpressionType.NESTED_DATA, ImmutableMap.of("x", 1L)).castTo(someComplex)
|
||||
);
|
||||
Assert.assertEquals("Invalid type, cannot cast [COMPLEX<json>] to [COMPLEX<tester>]", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -813,16 +1092,11 @@ public class EvalTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(type, eval.type());
|
||||
Assert.assertEquals(pair, eval.value());
|
||||
|
||||
// json type isn't defined in druid-core
|
||||
ExpressionType json = ExpressionType.fromString("COMPLEX<json>");
|
||||
eval = ExprEval.ofType(json, ImmutableMap.of("x", 1L, "y", 2L));
|
||||
Assert.assertEquals(json, eval.type());
|
||||
// json type best efforts its way to other types
|
||||
eval = ExprEval.ofType(ExpressionType.NESTED_DATA, ImmutableMap.of("x", 1L, "y", 2L));
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
Assert.assertEquals(ImmutableMap.of("x", 1L, "y", 2L), eval.value());
|
||||
|
||||
eval = ExprEval.ofType(json, "hello");
|
||||
Assert.assertEquals(json, eval.type());
|
||||
Assert.assertEquals("hello", eval.value());
|
||||
|
||||
ExpressionType stringyComplexThing = ExpressionType.fromString("COMPLEX<somestringything>");
|
||||
eval = ExprEval.ofType(stringyComplexThing, "notbase64");
|
||||
Assert.assertEquals(stringyComplexThing, eval.type());
|
||||
|
|
|
@ -19,22 +19,23 @@
|
|||
|
||||
package org.apache.druid.math.expr;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.column.TypeStrategies;
|
||||
import org.apache.druid.segment.column.TypeStrategiesTest;
|
||||
import org.apache.druid.segment.column.TypeStrategy;
|
||||
import org.apache.druid.segment.nested.StructuredData;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.math.BigDecimal;
|
||||
|
@ -44,10 +45,10 @@ import java.util.Set;
|
|||
|
||||
public class FunctionTest extends InitializedNullHandlingTest
|
||||
{
|
||||
@Rule
|
||||
public ExpectedException expectedException = ExpectedException.none();
|
||||
private Expr.ObjectBinding bestEffortBindings;
|
||||
private Expr.ObjectBinding typedBindings;
|
||||
private Expr.ObjectBinding[] allBindings;
|
||||
|
||||
private Expr.ObjectBinding bindings;
|
||||
|
||||
@BeforeClass
|
||||
public static void setupClass()
|
||||
|
@ -56,33 +57,66 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
TypeStrategiesTest.NULLABLE_TEST_PAIR_TYPE.getComplexTypeName(),
|
||||
new TypeStrategiesTest.NullableLongPairTypeStrategy()
|
||||
);
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
ImmutableMap.Builder<String, Object> builder =
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("x", "foo")
|
||||
.put("y", 2)
|
||||
.put("z", 3.1)
|
||||
.put("d", 34.56D)
|
||||
.put("maxLong", Long.MAX_VALUE)
|
||||
.put("minLong", Long.MIN_VALUE)
|
||||
.put("f", 12.34F)
|
||||
.put("nan", Double.NaN)
|
||||
.put("inf", Double.POSITIVE_INFINITY)
|
||||
.put("-inf", Double.NEGATIVE_INFINITY)
|
||||
.put("o", 0)
|
||||
.put("od", 0D)
|
||||
.put("of", 0F)
|
||||
.put("a", new String[] {"foo", "bar", "baz", "foobar"})
|
||||
.put("b", new Long[] {1L, 2L, 3L, 4L, 5L})
|
||||
.put("c", new Double[] {3.1, 4.2, 5.3})
|
||||
.put("someComplex", new TypeStrategiesTest.NullableLongPair(1L, 2L))
|
||||
.put("str1", "v1")
|
||||
.put("str2", "v2");
|
||||
bindings = InputBindings.forMap(builder.build());
|
||||
ImmutableMap.Builder<String, ExpressionType> inputTypesBuilder = ImmutableMap.builder();
|
||||
inputTypesBuilder.put("x", ExpressionType.STRING)
|
||||
.put("y", ExpressionType.LONG)
|
||||
.put("z", ExpressionType.DOUBLE)
|
||||
.put("d", ExpressionType.DOUBLE)
|
||||
.put("maxLong", ExpressionType.LONG)
|
||||
.put("minLong", ExpressionType.LONG)
|
||||
.put("f", ExpressionType.DOUBLE)
|
||||
.put("nan", ExpressionType.DOUBLE)
|
||||
.put("inf", ExpressionType.DOUBLE)
|
||||
.put("-inf", ExpressionType.DOUBLE)
|
||||
.put("o", ExpressionType.LONG)
|
||||
.put("od", ExpressionType.DOUBLE)
|
||||
.put("of", ExpressionType.DOUBLE)
|
||||
.put("a", ExpressionType.STRING_ARRAY)
|
||||
.put("b", ExpressionType.LONG_ARRAY)
|
||||
.put("c", ExpressionType.DOUBLE_ARRAY)
|
||||
.put("someComplex", ExpressionType.fromColumnType(TypeStrategiesTest.NULLABLE_TEST_PAIR_TYPE))
|
||||
.put("str1", ExpressionType.STRING)
|
||||
.put("str2", ExpressionType.STRING)
|
||||
.put("nestedArray", ExpressionType.NESTED_DATA);
|
||||
|
||||
final StructuredData nestedArray = StructuredData.wrap(
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("x", 2L, "y", 3.3),
|
||||
ImmutableMap.of("x", 4L, "y", 6.6)
|
||||
)
|
||||
);
|
||||
ImmutableMap.Builder<String, Object> builder = ImmutableMap.builder();
|
||||
builder.put("x", "foo")
|
||||
.put("y", 2)
|
||||
.put("z", 3.1)
|
||||
.put("d", 34.56D)
|
||||
.put("maxLong", Long.MAX_VALUE)
|
||||
.put("minLong", Long.MIN_VALUE)
|
||||
.put("f", 12.34F)
|
||||
.put("nan", Double.NaN)
|
||||
.put("inf", Double.POSITIVE_INFINITY)
|
||||
.put("-inf", Double.NEGATIVE_INFINITY)
|
||||
.put("o", 0)
|
||||
.put("od", 0D)
|
||||
.put("of", 0F)
|
||||
.put("a", new String[]{"foo", "bar", "baz", "foobar"})
|
||||
.put("b", new Long[]{1L, 2L, 3L, 4L, 5L})
|
||||
.put("c", new Double[]{3.1, 4.2, 5.3})
|
||||
.put("someComplex", new TypeStrategiesTest.NullableLongPair(1L, 2L))
|
||||
.put("str1", "v1")
|
||||
.put("str2", "v2")
|
||||
.put("nestedArray", nestedArray);
|
||||
bestEffortBindings = InputBindings.forMap(builder.build());
|
||||
typedBindings = InputBindings.forMap(
|
||||
builder.build(), InputBindings.inspectorFromTypeMap(inputTypesBuilder.build())
|
||||
);
|
||||
allBindings = new Expr.ObjectBinding[]{bestEffortBindings, typedBindings};
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -243,6 +277,8 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
{
|
||||
assertExpr("array_length([1,2,3])", 3L);
|
||||
assertExpr("array_length(a)", 4L);
|
||||
// nested types only work with typed bindings right now, and pretty limited support for stuff
|
||||
assertExpr("array_length(nestedArray)", 2L, typedBindings);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -251,6 +287,8 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
assertExpr("array_offset([1, 2, 3], 2)", 3L);
|
||||
assertArrayExpr("array_offset([1, 2, 3], 3)", null);
|
||||
assertExpr("array_offset(a, 2)", "baz");
|
||||
// nested types only work with typed bindings right now, and pretty limited support for stuff
|
||||
assertExpr("array_offset(nestedArray, 1)", ImmutableMap.of("x", 4L, "y", 6.6), typedBindings);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -259,6 +297,8 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
assertExpr("array_ordinal([1, 2, 3], 3)", 3L);
|
||||
assertArrayExpr("array_ordinal([1, 2, 3], 4)", null);
|
||||
assertExpr("array_ordinal(a, 3)", "baz");
|
||||
// nested types only work with typed bindings right now, and pretty limited support for stuff
|
||||
assertExpr("array_ordinal(nestedArray, 2)", ImmutableMap.of("x", 4L, "y", 6.6), typedBindings);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -671,7 +711,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
try {
|
||||
//x = "foo"
|
||||
Parser.parse("human_readable_binary_byte_format(x)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
|
||||
// for sqlCompatible, function above returns null and goes here
|
||||
// but for non-sqlCompatible, it must not go to here
|
||||
|
@ -687,7 +727,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
try {
|
||||
//x = "foo"
|
||||
Parser.parse("human_readable_binary_byte_format(1024, x)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
|
||||
//must not go to here
|
||||
Assert.assertTrue(false);
|
||||
|
@ -702,7 +742,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
try {
|
||||
//of = 0F
|
||||
Parser.parse("human_readable_binary_byte_format(1024, of)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
|
||||
//must not go to here
|
||||
Assert.assertTrue(false);
|
||||
|
@ -717,7 +757,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
try {
|
||||
//of = 0F
|
||||
Parser.parse("human_readable_binary_byte_format(1024, nonexist)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
|
||||
//must not go to here
|
||||
Assert.assertTrue(false);
|
||||
|
@ -735,7 +775,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
{
|
||||
try {
|
||||
Parser.parse("human_readable_binary_byte_format(1024, maxLong)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
catch (ExpressionValidationException e) {
|
||||
|
@ -747,7 +787,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
|
||||
try {
|
||||
Parser.parse("human_readable_binary_byte_format(1024, minLong)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
catch (ExpressionValidationException e) {
|
||||
|
@ -759,7 +799,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
|
||||
try {
|
||||
Parser.parse("human_readable_binary_byte_format(1024, -1)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
catch (ExpressionValidationException e) {
|
||||
|
@ -771,7 +811,7 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
|
||||
try {
|
||||
Parser.parse("human_readable_binary_byte_format(1024, 4)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
.eval(bestEffortBindings);
|
||||
Assert.assertTrue(false);
|
||||
}
|
||||
catch (ExpressionValidationException e) {
|
||||
|
@ -785,10 +825,15 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testSizeFormatInvalidArgumentSize()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[human_readable_binary_byte_format] requires 1 or 2 arguments");
|
||||
Parser.parse("human_readable_binary_byte_format(1024, 2, 3)", ExprMacroTable.nil())
|
||||
.eval(bindings);
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> Parser.parse(
|
||||
"human_readable_binary_byte_format(1024, 2, 3)",
|
||||
ExprMacroTable.nil()
|
||||
).eval(bestEffortBindings)
|
||||
);
|
||||
|
||||
Assert.assertEquals("Function[human_readable_binary_byte_format] requires 1 or 2 arguments", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -915,50 +960,49 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testComplexDecodeBaseWrongArgCount()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[complex_decode_base64] requires 2 arguments");
|
||||
assertExpr(
|
||||
"complex_decode_base64(string)",
|
||||
null
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertExpr("complex_decode_base64(string)", null)
|
||||
);
|
||||
Assert.assertEquals("Function[complex_decode_base64] requires 2 arguments", t.getMessage());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComplexDecodeBaseArg0Null()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage(
|
||||
"Function[complex_decode_base64] first argument must be constant STRING expression containing a valid complex type name but got NULL instead"
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertExpr("complex_decode_base64(null, string)", null)
|
||||
);
|
||||
assertExpr(
|
||||
"complex_decode_base64(null, string)",
|
||||
null
|
||||
Assert.assertEquals(
|
||||
"Function[complex_decode_base64] first argument must be constant STRING expression containing a valid complex type name but got NULL instead",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComplexDecodeBaseArg0BadType()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage(
|
||||
"Function[complex_decode_base64] first argument must be constant STRING expression containing a valid complex type name but got '1' instead"
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertExpr("complex_decode_base64(1, string)", null)
|
||||
);
|
||||
assertExpr(
|
||||
"complex_decode_base64(1, string)",
|
||||
null
|
||||
Assert.assertEquals(
|
||||
"Function[complex_decode_base64] first argument must be constant STRING expression containing a valid complex type name but got '1' instead",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testComplexDecodeBaseArg0Unknown()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage(
|
||||
"Function[complex_decode_base64] first argument must be a valid COMPLEX type name, got unknown COMPLEX type [COMPLEX<unknown>]"
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertExpr("complex_decode_base64('unknown', string)", null)
|
||||
);
|
||||
assertExpr(
|
||||
"complex_decode_base64('unknown', string)",
|
||||
null
|
||||
Assert.assertEquals(
|
||||
"Function[complex_decode_base64] first argument must be a valid COMPLEX type name, got unknown COMPLEX type [COMPLEX<unknown>]",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -972,33 +1016,53 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMVToArrayWithConstantLiteral()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[mv_to_array] argument 1 should be an identifier expression. Use array() instead");
|
||||
assertArrayExpr("mv_to_array('1')", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertArrayExpr("mv_to_array('1')", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Function[mv_to_array] argument 1 should be an identifier expression. Use array() instead",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMVToArrayWithFunction()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[mv_to_array] argument (repeat [hello, 2]) should be an identifier expression. Use array() instead");
|
||||
assertArrayExpr("mv_to_array(repeat('hello', 2))", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertArrayExpr("mv_to_array(repeat('hello', 2))", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Function[mv_to_array] argument (repeat [hello, 2]) should be an identifier expression. Use array() instead",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMVToArrayWithMoreArgs()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[mv_to_array] requires 1 argument");
|
||||
assertArrayExpr("mv_to_array(x,y)", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertArrayExpr("mv_to_array(x,y)", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Function[mv_to_array] requires 1 argument",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMVToArrayWithNoArgs()
|
||||
{
|
||||
expectedException.expect(ExpressionValidationException.class);
|
||||
expectedException.expectMessage("Function[mv_to_array] requires 1 argument");
|
||||
assertArrayExpr("mv_to_array()", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
ExpressionValidationException.class,
|
||||
() -> assertArrayExpr("mv_to_array()", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Function[mv_to_array] requires 1 argument",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1010,28 +1074,54 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
@Test
|
||||
public void testMultiplyOnString()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage("operator '*' in expression (\"str1\" * \"str2\") is not supported on type STRING.");
|
||||
assertExpr("str1 * str2", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
IAE.class,
|
||||
() -> assertExpr("str1 * str2", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"operator '*' in expression (\"str1\" * \"str2\") is not supported on type STRING.",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMinusOnString()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage("operator '-' in expression (\"str1\" - \"str2\") is not supported on type STRING.");
|
||||
assertExpr("str1 - str2", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
IAE.class,
|
||||
() -> assertExpr("str1 - str2", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"operator '-' in expression (\"str1\" - \"str2\") is not supported on type STRING.",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDivOnString()
|
||||
{
|
||||
expectedException.expect(IAE.class);
|
||||
expectedException.expectMessage("operator '/' in expression (\"str1\" / \"str2\") is not supported on type STRING.");
|
||||
assertExpr("str1 / str2", null);
|
||||
Throwable t = Assert.assertThrows(
|
||||
IAE.class,
|
||||
() -> assertExpr("str1 / str2", null)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"operator '/' in expression (\"str1\" / \"str2\") is not supported on type STRING.",
|
||||
t.getMessage()
|
||||
);
|
||||
}
|
||||
|
||||
private void assertExpr(final String expression, @Nullable final Object expectedResult)
|
||||
{
|
||||
for (Expr.ObjectBinding toUse : allBindings) {
|
||||
assertExpr(expression, expectedResult, toUse);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertExpr(
|
||||
final String expression,
|
||||
@Nullable final Object expectedResult,
|
||||
Expr.ObjectBinding bindings
|
||||
)
|
||||
{
|
||||
final Expr expr = Parser.parse(expression, ExprMacroTable.nil());
|
||||
Assert.assertEquals(expression, expectedResult, expr.eval(bindings).value());
|
||||
|
@ -1050,6 +1140,18 @@ public class FunctionTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
private void assertArrayExpr(final String expression, @Nullable final Object[] expectedResult)
|
||||
{
|
||||
|
||||
for (Expr.ObjectBinding toUse : allBindings) {
|
||||
assertArrayExpr(expression, expectedResult, toUse);
|
||||
}
|
||||
}
|
||||
|
||||
private void assertArrayExpr(
|
||||
final String expression,
|
||||
@Nullable final Object[] expectedResult,
|
||||
Expr.ObjectBinding bindings
|
||||
)
|
||||
{
|
||||
final Expr expr = Parser.parse(expression, ExprMacroTable.nil());
|
||||
Assert.assertArrayEquals(expression, expectedResult, expr.eval(bindings).asArray());
|
||||
|
|
|
@ -260,6 +260,18 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest
|
|||
testExpression("concat(s1,'-',s2,'-',l1,'-',d1)", types);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstants()
|
||||
{
|
||||
testExpression("null", types);
|
||||
testExpression("1", types);
|
||||
testExpression("1.1", types);
|
||||
testExpression("NaN", types);
|
||||
testExpression("Infinity", types);
|
||||
testExpression("-Infinity", types);
|
||||
testExpression("'hello'", types);
|
||||
}
|
||||
|
||||
static void testFunctions(Map<String, ExpressionType> types, String[] templates, String[] args)
|
||||
{
|
||||
for (String template : templates) {
|
||||
|
|
|
@ -54,11 +54,13 @@ import org.apache.druid.query.QueryRunnerFactory;
|
|||
import org.apache.druid.query.QueryRunnerTestHelper;
|
||||
import org.apache.druid.query.QueryToolChest;
|
||||
import org.apache.druid.query.context.ResponseContext;
|
||||
import org.apache.druid.query.groupby.GroupByQuery;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.apache.druid.query.groupby.ResultRow;
|
||||
import org.apache.druid.query.groupby.TestGroupByBuffers;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
|
||||
import org.apache.druid.query.scan.ScanQueryConfig;
|
||||
import org.apache.druid.query.scan.ScanQueryEngine;
|
||||
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
|
||||
|
@ -100,6 +102,7 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This class provides general utility to test any druid aggregation implementation given raw data,
|
||||
|
@ -785,13 +788,30 @@ public class AggregationTestHelper implements Closeable
|
|||
);
|
||||
String resultStr = mapper.writer().writeValueAsString(yielder);
|
||||
|
||||
List resultRows = Lists.transform(
|
||||
List<ResultRow> resultRows = Lists.transform(
|
||||
readQueryResultArrayFromString(resultStr),
|
||||
toolChest.makePreComputeManipulatorFn(
|
||||
queryPlus.getQuery(),
|
||||
MetricManipulatorFns.deserializing()
|
||||
)
|
||||
);
|
||||
|
||||
// coerce stuff so merge happens right after serde
|
||||
if (queryPlus.getQuery() instanceof GroupByQuery) {
|
||||
List<ResultRow> comparable =
|
||||
resultRows.stream()
|
||||
.peek(row -> {
|
||||
GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
|
||||
GroupByQueryEngineV2.convertRowTypesToOutputTypes(
|
||||
query.getDimensions(),
|
||||
row,
|
||||
query.getResultRowDimensionStart()
|
||||
);
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return Sequences.simple(comparable);
|
||||
}
|
||||
return Sequences.simple(resultRows);
|
||||
}
|
||||
catch (Exception ex) {
|
||||
|
|
|
@ -66,9 +66,9 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
|
||||
Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers(
|
||||
new ImmutableMap.Builder<String, InputBindings.InputSupplier>()
|
||||
.put("nest", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> NEST))
|
||||
.put("nestWrapped", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> new StructuredData(NEST)))
|
||||
.put("nester", InputBindings.inputSupplier(NestedDataExpressions.TYPE, () -> NESTER))
|
||||
.put("nest", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NEST))
|
||||
.put("nestWrapped", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> new StructuredData(NEST)))
|
||||
.put("nester", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NESTER))
|
||||
.put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> "abcdef"))
|
||||
.put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> 1234L))
|
||||
.put("double", InputBindings.inputSupplier(ExpressionType.DOUBLE, () -> 1.234))
|
||||
|
@ -209,17 +209,17 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
Expr expr = Parser.parse("json_query(nest, '$.x')", MACRO_TABLE);
|
||||
ExprEval eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(100L, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("json_query(nester, '$.x')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(NESTER.get("x"), eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("json_query(nester, '$.x[1]')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("b", eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("json_query(nester, '$.x[23]')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
|
@ -236,7 +236,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
expr = Parser.parse("json_query(nester, '$.y.a')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("hello", eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("json_query(nester, '$.y.a.b.c[12]')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
|
@ -245,7 +245,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
expr = Parser.parse("json_query(long, '$')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(1234L, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -254,44 +254,44 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
Expr expr = Parser.parse("parse_json(null)", MACRO_TABLE);
|
||||
ExprEval eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('null')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
Assert.assertThrows(ExpressionProcessingException.class, () -> Parser.parse("parse_json('{')", MACRO_TABLE));
|
||||
expr = Parser.parse("try_parse_json('{')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
Assert.assertThrows(ExpressionProcessingException.class, () -> Parser.parse("parse_json('hello world')", MACRO_TABLE));
|
||||
expr = Parser.parse("try_parse_json('hello world')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(null, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('\"hello world\"')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("hello world", eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('1')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(1, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('true')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals(true, eval.value());
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("parse_json('{\"foo\":1}')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
Assert.assertEquals("{\"foo\":1}", JSON_MAPPER.writeValueAsString(eval.value()));
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -309,7 +309,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
|
|||
Map val = (Map) eval.value();
|
||||
Assert.assertEquals(NEST.get(key), ((Integer) val.get(key)).longValue());
|
||||
}
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, eval.type());
|
||||
|
||||
expr = Parser.parse("json_value(parse_json('{\"x\":100,\"y\":200,\"z\":300}'), '$.x')", MACRO_TABLE);
|
||||
eval = expr.eval(inputBindings);
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.filter.vector;
|
||||
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.segment.IdLookup;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
|
@ -63,13 +62,8 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul
|
|||
Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
|
||||
|
||||
// in default mode, matching null produces a boolean matcher
|
||||
VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
} else {
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
}
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
|
||||
EasyMock.verify(vectorValueSelector);
|
||||
|
@ -92,13 +86,9 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul
|
|||
Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
|
||||
|
||||
// in default mode, matching null produces a boolean matcher
|
||||
|
||||
VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
} else {
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
}
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
|
||||
EasyMock.verify(vectorValueSelector);
|
||||
|
@ -120,13 +110,8 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul
|
|||
Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize());
|
||||
|
||||
// in default mode, matching null produces a boolean matcher
|
||||
VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
} else {
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
}
|
||||
Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher);
|
||||
Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize());
|
||||
Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize());
|
||||
EasyMock.verify(vectorValueSelector);
|
||||
|
|
|
@ -368,7 +368,7 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBySomeFieldOnStringColumnWithFilterExpectedType()
|
||||
public void testGroupBySomeFieldOnStringColumnWithFilterExpectedTypeLong()
|
||||
{
|
||||
List<String> vals = new ArrayList<>();
|
||||
vals.add("100");
|
||||
|
@ -396,6 +396,64 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBySomeFieldOnStringColumnWithFilterExpectedTypeDouble()
|
||||
{
|
||||
List<String> vals = new ArrayList<>();
|
||||
vals.add("100");
|
||||
vals.add("200");
|
||||
vals.add("300");
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.DOUBLE))
|
||||
.setVirtualColumns(new NestedFieldVirtualColumn("dim", "$", "v0", ColumnType.LONG))
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.setDimFilter(new InDimFilter("v0", vals, null))
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{100.0, 2L}
|
||||
),
|
||||
false,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBySomeFieldOnStringColumnWithFilterExpectedTypeFloat()
|
||||
{
|
||||
List<String> vals = new ArrayList<>();
|
||||
vals.add("100");
|
||||
vals.add("200");
|
||||
vals.add("300");
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.FLOAT))
|
||||
.setVirtualColumns(new NestedFieldVirtualColumn("dim", "$", "v0", ColumnType.LONG))
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.setDimFilter(new InDimFilter("v0", vals, null))
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{100f, 2L}
|
||||
),
|
||||
false,
|
||||
true
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupBySomeFieldOnStringColumnWithFilterNil()
|
||||
{
|
||||
|
@ -603,8 +661,10 @@ public class NestedDataGroupByQueryTest extends InitializedNullHandlingTest
|
|||
final Object[] resultRow = results.get(i).getArray();
|
||||
Assert.assertEquals(expected.get(i).length, resultRow.length);
|
||||
for (int j = 0; j < resultRow.length; j++) {
|
||||
if (rowSignature.getColumnType(j).map(t -> t.anyOf(ValueType.DOUBLE, ValueType.FLOAT)).orElse(false)) {
|
||||
if (rowSignature.getColumnType(j).map(t -> t.is(ValueType.DOUBLE)).orElse(false)) {
|
||||
Assert.assertEquals((Double) expected.get(i)[j], (Double) resultRow[j], 0.01);
|
||||
} else if (rowSignature.getColumnType(j).map(t -> t.is(ValueType.FLOAT)).orElse(false)) {
|
||||
Assert.assertEquals((Float) expected.get(i)[j], (Float) resultRow[j], 0.01);
|
||||
} else {
|
||||
Assert.assertEquals(expected.get(i)[j], resultRow[j]);
|
||||
}
|
||||
|
|
|
@ -0,0 +1,505 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query.groupby;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.query.NestedDataTestUtils;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
|
||||
import org.apache.druid.query.groupby.strategy.GroupByStrategySelector;
|
||||
import org.apache.druid.segment.Segment;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.ComparableList;
|
||||
import org.apache.druid.segment.data.ComparableStringArray;
|
||||
import org.apache.druid.segment.virtual.NestedFieldVirtualColumn;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class NestedGroupByArrayQueryTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(NestedDataGroupByQueryTest.class);
|
||||
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
|
||||
|
||||
@Rule
|
||||
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||
|
||||
private final Closer closer;
|
||||
private final GroupByQueryConfig config;
|
||||
private final QueryContexts.Vectorize vectorize;
|
||||
private final AggregationTestHelper helper;
|
||||
private final BiFunction<TemporaryFolder, Closer, List<Segment>> segmentsGenerator;
|
||||
private final String segmentsName;
|
||||
|
||||
public NestedGroupByArrayQueryTest(
|
||||
GroupByQueryConfig config,
|
||||
BiFunction<TemporaryFolder, Closer, List<Segment>> segmentGenerator,
|
||||
String vectorize
|
||||
)
|
||||
{
|
||||
NestedDataModule.registerHandlersAndSerde();
|
||||
this.config = config;
|
||||
this.vectorize = QueryContexts.Vectorize.fromString(vectorize);
|
||||
this.helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
NestedDataModule.getJacksonModulesList(),
|
||||
config,
|
||||
tempFolder
|
||||
);
|
||||
this.segmentsGenerator = segmentGenerator;
|
||||
this.segmentsName = segmentGenerator.toString();
|
||||
this.closer = Closer.create();
|
||||
}
|
||||
|
||||
public Map<String, Object> getContext()
|
||||
{
|
||||
return ImmutableMap.of(
|
||||
QueryContexts.VECTORIZE_KEY, vectorize.toString(),
|
||||
QueryContexts.VECTORIZE_VIRTUAL_COLUMNS_KEY, vectorize.toString()
|
||||
);
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "config = {0}, segments = {1}, vectorize = {2}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
final List<BiFunction<TemporaryFolder, Closer, List<Segment>>> segmentsGenerators =
|
||||
NestedDataTestUtils.getSegmentGenerators(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE);
|
||||
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
if (GroupByStrategySelector.STRATEGY_V1.equals(config.getDefaultStrategy())) {
|
||||
// group by v1 doesn't support array stuff
|
||||
continue;
|
||||
}
|
||||
for (BiFunction<TemporaryFolder, Closer, List<Segment>> generatorFn : segmentsGenerators) {
|
||||
// skip force because arrays don't really support vectorize engine, but we want the coverage for once they do...
|
||||
for (String vectorize : new String[]{"false", "true"}) {
|
||||
constructors.add(new Object[]{config, generatorFn, vectorize});
|
||||
}
|
||||
}
|
||||
}
|
||||
return constructors;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup()
|
||||
{
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws IOException
|
||||
{
|
||||
closer.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayString()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("arrayString", ColumnType.STRING_ARRAY))
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, 8L},
|
||||
new Object[]{ComparableStringArray.of("a", "b"), 8L},
|
||||
new Object[]{ComparableStringArray.of("a", "b", "c"), 4L},
|
||||
new Object[]{ComparableStringArray.of("b", "c"), 4L},
|
||||
new Object[]{ComparableStringArray.of("d", "e"), 4L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayLong()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("arrayLong", ColumnType.LONG_ARRAY))
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, 8L},
|
||||
new Object[]{asComparableList(1L, 2L, 3L), 8L},
|
||||
new Object[]{asComparableList(1L, 2L, 3L, 4L), 4L},
|
||||
new Object[]{asComparableList(1L, 4L), 4L},
|
||||
new Object[]{asComparableList(2L, 3L), 4L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayDouble()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("arrayDouble", ColumnType.DOUBLE_ARRAY))
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, 8L},
|
||||
new Object[]{asComparableList(1.1, 2.2, 3.3), 8L},
|
||||
new Object[]{asComparableList(1.1, 3.3), 4L},
|
||||
new Object[]{asComparableList(2.2, 3.3, 4.0), 4L},
|
||||
new Object[]{asComparableList(3.3, 4.4, 5.5), 4L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayStringElement()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.STRING))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayString",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.STRING
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, 24L},
|
||||
new Object[]{"c", 4L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayStringElementDouble()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.DOUBLE))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayString",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.DOUBLE
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultDoubleValue(), 28L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayStringElementLong()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.LONG))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayString",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.LONG
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultLongValue(), 28L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayStringElementFloat()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.FLOAT))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayString",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.FLOAT
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultFloatValue(), 28L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayLongElement()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.LONG))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayLong",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.LONG
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultLongValue(), 16L},
|
||||
new Object[]{3L, 12L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayLongElementDouble()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.DOUBLE))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayLong",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.DOUBLE
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultDoubleValue(), 16L},
|
||||
new Object[]{3.0, 12L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayLongElementFloat()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.FLOAT))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayLong",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.FLOAT
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultFloatValue(), 16L},
|
||||
new Object[]{3.0f, 12L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootArrayLongElementString()
|
||||
{
|
||||
GroupByQuery groupQuery = GroupByQuery.builder()
|
||||
.setDataSource("test_datasource")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(Intervals.ETERNITY)
|
||||
.setDimensions(DefaultDimensionSpec.of("v0", ColumnType.STRING))
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn(
|
||||
"arrayLong",
|
||||
"$[2]",
|
||||
"v0",
|
||||
ColumnType.STRING
|
||||
)
|
||||
)
|
||||
.setAggregatorSpecs(new CountAggregatorFactory("count"))
|
||||
.setContext(getContext())
|
||||
.build();
|
||||
|
||||
|
||||
runResults(
|
||||
groupQuery,
|
||||
ImmutableList.of(
|
||||
new Object[]{null, 16L},
|
||||
new Object[]{"3", 12L}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
private void runResults(
|
||||
GroupByQuery groupQuery,
|
||||
List<Object[]> expectedResults
|
||||
)
|
||||
{
|
||||
Supplier<List<ResultRow>> runner =
|
||||
() -> helper.runQueryOnSegmentsObjs(segmentsGenerator.apply(tempFolder, closer), groupQuery).toList();
|
||||
|
||||
List<ResultRow> results = runner.get();
|
||||
verifyResults(
|
||||
groupQuery,
|
||||
results,
|
||||
expectedResults
|
||||
);
|
||||
}
|
||||
|
||||
private static void verifyResults(GroupByQuery query, List<ResultRow> results, List<Object[]> expected)
|
||||
{
|
||||
RowSignature rowSignature = query.getResultRowSignature();
|
||||
List<ResultRow> serdeAndBack =
|
||||
results.stream()
|
||||
.peek(
|
||||
row -> GroupByQueryEngineV2.convertRowTypesToOutputTypes(
|
||||
query.getDimensions(),
|
||||
row,
|
||||
query.getResultRowDimensionStart()
|
||||
)
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
LOG.info("results:\n%s", serdeAndBack.stream().map(ResultRow::toString).collect(Collectors.joining("\n")));
|
||||
Assert.assertEquals(expected.size(), serdeAndBack.size());
|
||||
for (int i = 0; i < expected.size(); i++) {
|
||||
final Object[] resultRow = serdeAndBack.get(i).getArray();
|
||||
Assert.assertEquals(expected.get(i).length, resultRow.length);
|
||||
for (int j = 0; j < resultRow.length; j++) {
|
||||
if (expected.get(i)[j] == null) {
|
||||
Assert.assertNull(resultRow[j]);
|
||||
} else if (rowSignature.getColumnType(j).map(t -> t.is(ValueType.DOUBLE)).orElse(false)) {
|
||||
Assert.assertEquals((Double) expected.get(i)[j], (Double) resultRow[j], 0.01);
|
||||
} else if (rowSignature.getColumnType(j).map(t -> t.is(ValueType.FLOAT)).orElse(false)) {
|
||||
Assert.assertEquals((Float) expected.get(i)[j], (Float) resultRow[j], 0.01);
|
||||
} else {
|
||||
Assert.assertEquals(expected.get(i)[j], resultRow[j]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static <T extends Comparable> ComparableList<T> asComparableList(T... objects)
|
||||
{
|
||||
return new ComparableList<>(Arrays.asList(objects));
|
||||
}
|
||||
}
|
|
@ -188,7 +188,9 @@ public class NestedDataScanQueryTest extends InitializedNullHandlingTest
|
|||
logResults(resultsRealtime);
|
||||
Assert.assertEquals(1, resultsRealtime.size());
|
||||
Assert.assertEquals(resultsRealtime.size(), resultsSegments.size());
|
||||
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertEquals(resultsSegments.get(0).getEvents().toString(), resultsRealtime.get(0).getEvents().toString());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -68,60 +68,70 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
|
|||
public void testKeySizeEstimation()
|
||||
{
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
Assert.assertEquals(0, indexer.getCardinality());
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
// new raw value, new field, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
|
||||
Assert.assertEquals(228, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(1, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
// adding same value only adds estimated size of value itself
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
|
||||
Assert.assertEquals(112, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(1, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
// new raw value, new field, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
|
||||
Assert.assertEquals(94, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(2, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
|
||||
// adding same value only adds estimated size of value itself
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(2, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
|
||||
// new raw value, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(11L, false);
|
||||
Assert.assertEquals(48, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(3, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 3, indexer.getCardinality());
|
||||
|
||||
// new raw value, new fields
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
|
||||
Assert.assertEquals(168, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
// new raw value, re-use fields and dictionary
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
|
||||
Assert.assertEquals(104, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
// new raw value, new fields
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
|
||||
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
|
||||
false
|
||||
);
|
||||
Assert.assertEquals(166, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
// new raw value
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
|
||||
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
|
||||
false
|
||||
);
|
||||
Assert.assertEquals(166, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(0, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(7, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
|
||||
} else {
|
||||
Assert.assertEquals(104, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(7, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
|
||||
}
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(0L, false);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
|
||||
} else {
|
||||
Assert.assertEquals(48, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 8, indexer.getCardinality());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -247,20 +257,43 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
|
|||
columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultLongValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject());
|
||||
}
|
||||
|
||||
|
||||
columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultLongValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -320,18 +353,42 @@ public class AutoTypeColumnIndexerTest extends InitializedNullHandlingTest
|
|||
columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultDoubleValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject());
|
||||
}
|
||||
|
||||
columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultDoubleValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -57,6 +57,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Helps tests make segments.
|
||||
|
@ -97,7 +98,7 @@ public class IndexBuilder
|
|||
|
||||
public static IndexBuilder create()
|
||||
{
|
||||
return new IndexBuilder(TestHelper.JSON_MAPPER, TestHelper.NO_CACHE_COLUMN_CONFIG);
|
||||
return new IndexBuilder(TestHelper.JSON_MAPPER, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
}
|
||||
|
||||
public static IndexBuilder create(ColumnConfig columnConfig)
|
||||
|
@ -107,7 +108,7 @@ public class IndexBuilder
|
|||
|
||||
public static IndexBuilder create(ObjectMapper jsonMapper)
|
||||
{
|
||||
return new IndexBuilder(jsonMapper, TestHelper.NO_CACHE_COLUMN_CONFIG);
|
||||
return new IndexBuilder(jsonMapper, TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
}
|
||||
|
||||
public static IndexBuilder create(ObjectMapper jsonMapper, ColumnConfig columnConfig)
|
||||
|
@ -203,6 +204,12 @@ public class IndexBuilder
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder mapSchema(Function<IncrementalIndexSchema, IncrementalIndexSchema> f)
|
||||
{
|
||||
this.schema = f.apply(this.schema);
|
||||
return this;
|
||||
}
|
||||
|
||||
public IncrementalIndex buildIncrementalIndex()
|
||||
{
|
||||
if (inputSource != null) {
|
||||
|
@ -259,6 +266,7 @@ public class IndexBuilder
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
public QueryableIndex buildMMappedMergedIndex()
|
||||
{
|
||||
Preconditions.checkNotNull(tmpDir, "tmpDir");
|
||||
|
|
|
@ -118,7 +118,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling
|
|||
@Test
|
||||
public void testV9LoaderThatIgnoresmptyColumns() throws IOException
|
||||
{
|
||||
QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(TestHelper.NO_CACHE_COLUMN_CONFIG).load(
|
||||
QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG).load(
|
||||
segmentDir,
|
||||
TestHelper.makeJsonMapper(),
|
||||
false,
|
||||
|
|
|
@ -68,60 +68,70 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
|||
public void testKeySizeEstimation()
|
||||
{
|
||||
NestedDataColumnIndexer indexer = new NestedDataColumnIndexer();
|
||||
Assert.assertEquals(0, indexer.getCardinality());
|
||||
int baseCardinality = NullHandling.sqlCompatible() ? 0 : 2;
|
||||
Assert.assertEquals(baseCardinality, indexer.getCardinality());
|
||||
|
||||
EncodedKeyComponent<StructuredData> key;
|
||||
// new raw value, new field, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
|
||||
Assert.assertEquals(228, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(1, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
// adding same value only adds estimated size of value itself
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableMap.of("x", "foo"), false);
|
||||
Assert.assertEquals(112, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(1, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 1, indexer.getCardinality());
|
||||
// new raw value, new field, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
|
||||
Assert.assertEquals(94, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(2, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
|
||||
// adding same value only adds estimated size of value itself
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(10L, false);
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(2, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 2, indexer.getCardinality());
|
||||
// new raw value, new dictionary entry
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(11L, false);
|
||||
Assert.assertEquals(48, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(3, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 3, indexer.getCardinality());
|
||||
|
||||
// new raw value, new fields
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
|
||||
Assert.assertEquals(276, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(5, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 5, indexer.getCardinality());
|
||||
// new raw value, re-use fields and dictionary
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(ImmutableList.of(1L, 2L, 10L), false);
|
||||
Assert.assertEquals(56, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(5, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 5, indexer.getCardinality());
|
||||
// new raw value, new fields
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
|
||||
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
|
||||
false
|
||||
);
|
||||
Assert.assertEquals(286, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(5, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 5, indexer.getCardinality());
|
||||
// new raw value
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(
|
||||
ImmutableMap.of("x", ImmutableList.of(1L, 2L, 10L)),
|
||||
false
|
||||
);
|
||||
Assert.assertEquals(118, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(5, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 5, indexer.getCardinality());
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent("", false);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(0, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
} else {
|
||||
Assert.assertEquals(104, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(6, indexer.getCardinality());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
}
|
||||
|
||||
key = indexer.processRowValsToUnsortedEncodedKeyComponent(0, false);
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
Assert.assertEquals(16, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 6, indexer.getCardinality());
|
||||
} else {
|
||||
Assert.assertEquals(48, key.getEffectiveSizeBytes());
|
||||
Assert.assertEquals(baseCardinality + 7, indexer.getCardinality());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -247,20 +257,42 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
|||
columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultLongValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject());
|
||||
}
|
||||
|
||||
columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(LONG_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultLongValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultLongValue()), dimensionSelector.getObject());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -320,18 +352,42 @@ public class NestedDataColumnIndexerTest extends InitializedNullHandlingTest
|
|||
columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultDoubleValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject());
|
||||
}
|
||||
|
||||
columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory();
|
||||
valueSelector = columnSelectorFactory.makeColumnValueSelector(DOUBLE_COL);
|
||||
dimensionSelector = columnSelectorFactory.makeDimensionSelector(dimensionSpec);
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertNull(dimensionSelector.getObject());
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(1, dimensionSelector.getRow().size());
|
||||
Assert.assertEquals(
|
||||
String.valueOf(NullHandling.defaultDoubleValue()),
|
||||
dimensionSelector.lookupName(dimensionSelector.getRow().get(0))
|
||||
);
|
||||
Assert.assertEquals(String.valueOf(NullHandling.defaultDoubleValue()), dimensionSelector.getObject());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -62,7 +62,26 @@ import java.util.stream.IntStream;
|
|||
public class TestHelper
|
||||
{
|
||||
public static final ObjectMapper JSON_MAPPER = makeJsonMapper();
|
||||
public static final ColumnConfig NO_CACHE_COLUMN_CONFIG = () -> 0;
|
||||
public static final ColumnConfig NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG = new ColumnConfig()
|
||||
{
|
||||
@Override
|
||||
public int columnCacheSizeBytes()
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValueRangeIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double skipValuePredicateIndexScale()
|
||||
{
|
||||
return 1.0;
|
||||
}
|
||||
};
|
||||
|
||||
public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory)
|
||||
{
|
||||
|
@ -71,7 +90,7 @@ public class TestHelper
|
|||
|
||||
public static IndexIO getTestIndexIO()
|
||||
{
|
||||
return getTestIndexIO(NO_CACHE_COLUMN_CONFIG);
|
||||
return getTestIndexIO(NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG);
|
||||
}
|
||||
|
||||
public static IndexIO getTestIndexIO(ColumnConfig columnConfig)
|
||||
|
|
|
@ -145,6 +145,28 @@ public class ColumnTypeTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeastRestrictiveType()
|
||||
{
|
||||
Assert.assertEquals(ColumnType.STRING, ColumnType.leastRestrictiveType(ColumnType.STRING, ColumnType.DOUBLE));
|
||||
Assert.assertEquals(ColumnType.STRING, ColumnType.leastRestrictiveType(ColumnType.STRING, ColumnType.LONG));
|
||||
Assert.assertEquals(ColumnType.STRING, ColumnType.leastRestrictiveType(ColumnType.STRING, ColumnType.FLOAT));
|
||||
Assert.assertEquals(ColumnType.DOUBLE, ColumnType.leastRestrictiveType(ColumnType.FLOAT, ColumnType.DOUBLE));
|
||||
Assert.assertEquals(ColumnType.DOUBLE, ColumnType.leastRestrictiveType(ColumnType.FLOAT, ColumnType.LONG));
|
||||
Assert.assertEquals(ColumnType.LONG, ColumnType.leastRestrictiveType(ColumnType.LONG, ColumnType.LONG));
|
||||
Assert.assertEquals(ColumnType.LONG_ARRAY, ColumnType.leastRestrictiveType(ColumnType.LONG, ColumnType.LONG_ARRAY));
|
||||
Assert.assertEquals(ColumnType.STRING_ARRAY, ColumnType.leastRestrictiveType(ColumnType.STRING_ARRAY, ColumnType.LONG_ARRAY));
|
||||
Assert.assertEquals(ColumnType.STRING_ARRAY, ColumnType.leastRestrictiveType(ColumnType.STRING_ARRAY, ColumnType.DOUBLE_ARRAY));
|
||||
Assert.assertEquals(ColumnType.DOUBLE_ARRAY, ColumnType.leastRestrictiveType(ColumnType.LONG_ARRAY, ColumnType.DOUBLE_ARRAY));
|
||||
Assert.assertEquals(ColumnType.NESTED_DATA, ColumnType.leastRestrictiveType(ColumnType.STRING_ARRAY, ColumnType.NESTED_DATA));
|
||||
Assert.assertEquals(ColumnType.NESTED_DATA, ColumnType.leastRestrictiveType(ColumnType.NESTED_DATA, ColumnType.STRING_ARRAY));
|
||||
Assert.assertEquals(ColumnType.NESTED_DATA, ColumnType.leastRestrictiveType(ColumnType.NESTED_DATA, ColumnType.UNKNOWN_COMPLEX));
|
||||
Assert.assertEquals(SOME_COMPLEX, ColumnType.leastRestrictiveType(SOME_COMPLEX, ColumnType.UNKNOWN_COMPLEX));
|
||||
Assert.assertEquals(SOME_COMPLEX, ColumnType.leastRestrictiveType(ColumnType.UNKNOWN_COMPLEX, SOME_COMPLEX));
|
||||
Assert.assertThrows(IllegalArgumentException.class, () -> ColumnType.leastRestrictiveType(ColumnType.NESTED_DATA, SOME_COMPLEX));
|
||||
Assert.assertThrows(IllegalArgumentException.class, () -> ColumnType.leastRestrictiveType(ColumnType.STRING_ARRAY, SOME_COMPLEX));
|
||||
}
|
||||
|
||||
static class SomeOtherTypeSignature extends BaseTypeSignature<ValueType>
|
||||
{
|
||||
public SomeOtherTypeSignature(
|
||||
|
|
|
@ -64,6 +64,7 @@ import org.apache.druid.query.filter.DimFilter;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.filter.vector.VectorValueMatcher;
|
||||
import org.apache.druid.segment.AutoTypeColumnSchema;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.ColumnSelector;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
|
@ -119,6 +120,7 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
||||
{
|
||||
|
@ -310,44 +312,105 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
|
||||
final Map<String, Function<IndexBuilder, Pair<StorageAdapter, Closeable>>> finishers =
|
||||
ImmutableMap.<String, Function<IndexBuilder, Pair<StorageAdapter, Closeable>>>builder()
|
||||
.put(
|
||||
"incremental",
|
||||
input -> {
|
||||
final IncrementalIndex index = input.buildIncrementalIndex();
|
||||
return Pair.of(new IncrementalIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmapped",
|
||||
input -> {
|
||||
final QueryableIndex index = input.buildMMappedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmappedMerged",
|
||||
input -> {
|
||||
final QueryableIndex index = input.buildMMappedMergedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"rowBasedWithoutTypeSignature",
|
||||
input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> {})
|
||||
)
|
||||
.put(
|
||||
"rowBasedWithTypeSignature",
|
||||
input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {})
|
||||
)
|
||||
.put("frame (row-based)", input -> {
|
||||
final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED);
|
||||
return Pair.of(segment.asStorageAdapter(), segment);
|
||||
})
|
||||
.put("frame (columnar)", input -> {
|
||||
final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR);
|
||||
return Pair.of(segment.asStorageAdapter(), segment);
|
||||
})
|
||||
.build();
|
||||
.put(
|
||||
"incremental",
|
||||
input -> {
|
||||
final IncrementalIndex index = input.buildIncrementalIndex();
|
||||
return Pair.of(new IncrementalIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmappedAutoTypes",
|
||||
input -> {
|
||||
input.mapSchema(
|
||||
schema ->
|
||||
new IncrementalIndexSchema(
|
||||
schema.getMinTimestamp(),
|
||||
schema.getTimestampSpec(),
|
||||
schema.getGran(),
|
||||
schema.getVirtualColumns(),
|
||||
schema.getDimensionsSpec().withDimensions(
|
||||
schema.getDimensionsSpec()
|
||||
.getDimensions()
|
||||
.stream()
|
||||
.map(
|
||||
dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName())
|
||||
)
|
||||
.collect(Collectors.toList())
|
||||
),
|
||||
schema.getMetrics(),
|
||||
schema.isRollup()
|
||||
)
|
||||
);
|
||||
final QueryableIndex index = input.buildMMappedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmappedAutoTypesMerged",
|
||||
input -> {
|
||||
input.mapSchema(
|
||||
schema ->
|
||||
new IncrementalIndexSchema(
|
||||
schema.getMinTimestamp(),
|
||||
schema.getTimestampSpec(),
|
||||
schema.getGran(),
|
||||
schema.getVirtualColumns(),
|
||||
schema.getDimensionsSpec().withDimensions(
|
||||
schema.getDimensionsSpec()
|
||||
.getDimensions()
|
||||
.stream()
|
||||
.map(
|
||||
dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName())
|
||||
)
|
||||
.collect(Collectors.toList())
|
||||
),
|
||||
schema.getMetrics(),
|
||||
schema.isRollup()
|
||||
)
|
||||
);
|
||||
// if 1 row per segment some of the columns have null values for the row which causes 'auto'
|
||||
// typing default value coercion to be lost in default value mode, so make sure there is at
|
||||
// least one number in each segment for these tests to pass correctly because the column
|
||||
// is typeless and so doesn't write out zeros like regular numbers do
|
||||
input.intermediaryPersistSize(3);
|
||||
final QueryableIndex index = input.buildMMappedMergedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmapped",
|
||||
input -> {
|
||||
final QueryableIndex index = input.buildMMappedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"mmappedMerged",
|
||||
input -> {
|
||||
final QueryableIndex index = input.buildMMappedMergedIndex();
|
||||
return Pair.of(new QueryableIndexStorageAdapter(index), index);
|
||||
}
|
||||
)
|
||||
.put(
|
||||
"rowBasedWithoutTypeSignature",
|
||||
input -> Pair.of(input.buildRowBasedSegmentWithoutTypeSignature().asStorageAdapter(), () -> {
|
||||
})
|
||||
)
|
||||
.put(
|
||||
"rowBasedWithTypeSignature",
|
||||
input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {
|
||||
})
|
||||
)
|
||||
.put("frame (row-based)", input -> {
|
||||
final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED);
|
||||
return Pair.of(segment.asStorageAdapter(), segment);
|
||||
})
|
||||
.put("frame (columnar)", input -> {
|
||||
final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR);
|
||||
return Pair.of(segment.asStorageAdapter(), segment);
|
||||
})
|
||||
.build();
|
||||
|
||||
StringEncodingStrategy[] stringEncoding = new StringEncodingStrategy[]{
|
||||
new StringEncodingStrategy.Utf8(),
|
||||
|
@ -816,58 +879,66 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
|
|||
final boolean testVectorized
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(
|
||||
"Cursor: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilter(filter, "dim0")
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
try {
|
||||
Assert.assertEquals(
|
||||
"Cursor (vectorized): " + filter,
|
||||
"Cursor: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorCursor(filter, "dim0")
|
||||
selectColumnValuesMatchingFilter(filter, "dim0")
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Cursor (vectorized): " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorCursor(filter, "dim0")
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cursor Virtual Column (vectorized): " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorVirtualColumnCursor(filter, "vdim0", "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cursor Virtual Column (vectorized): " + filter,
|
||||
"Cursor with postFiltering: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorVirtualColumnCursor(filter, "vdim0", "dim0")
|
||||
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
|
||||
);
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering (vectorized): " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorizedPostFiltering(filter, "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering (vectorized): " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingVectorizedPostFiltering(filter, "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator: " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingFilteredAggregator(filter)
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator (vectorized): " + filter,
|
||||
"Filtered aggregator: " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingVectorizedFilteredAggregator(filter)
|
||||
selectCountUsingFilteredAggregator(filter)
|
||||
);
|
||||
|
||||
if (testVectorized) {
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator (vectorized): " + filter,
|
||||
expectedRows.size(),
|
||||
selectCountUsingVectorizedFilteredAggregator(filter)
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"RowBasedColumnSelectorFactory: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0")
|
||||
);
|
||||
}
|
||||
|
||||
Assert.assertEquals(
|
||||
"RowBasedColumnSelectorFactory: " + filter,
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0")
|
||||
);
|
||||
catch (ISE ise) {
|
||||
// ignore failures resulting from 'auto'
|
||||
if (!(testName.contains("AutoTypes") && "Unsupported type[ARRAY<STRING>]".equals(ise.getMessage()))) {
|
||||
throw ise;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -95,6 +95,11 @@ public class ColumnComparisonFilterTest extends BaseFilterTest
|
|||
@Test
|
||||
public void testColumnsWithoutNulls()
|
||||
{
|
||||
// columns have mixed type input and so are ingested as COMPLEX<json>
|
||||
// however the comparison filter currently nulls out complex types instead of comparing the values
|
||||
if (testName.contains("AutoTypes")) {
|
||||
return;
|
||||
}
|
||||
assertFilterMatchesSkipVectorize(new ColumnComparisonDimFilter(ImmutableList.of(
|
||||
DefaultDimensionSpec.of("dim0"),
|
||||
DefaultDimensionSpec.of("dim1")
|
||||
|
@ -117,6 +122,11 @@ public class ColumnComparisonFilterTest extends BaseFilterTest
|
|||
@Test
|
||||
public void testMissingColumnNotSpecifiedInDimensionList()
|
||||
{
|
||||
// columns have mixed type input and so are ingested as COMPLEX<json>
|
||||
// however the comparison filter currently nulls out complex types instead of comparing the values
|
||||
if (testName.contains("AutoTypes")) {
|
||||
return;
|
||||
}
|
||||
assertFilterMatchesSkipVectorize(new ColumnComparisonDimFilter(ImmutableList.of(
|
||||
DefaultDimensionSpec.of("dim6"),
|
||||
DefaultDimensionSpec.of("dim7")
|
||||
|
|
|
@ -162,6 +162,10 @@ public class ExpressionFilterTest extends BaseFilterTest
|
|||
@Test
|
||||
public void testOneMultiValuedStringColumn()
|
||||
{
|
||||
// auto type columns don't support mvds, bail out
|
||||
if (testName.contains("AutoTypes")) {
|
||||
return;
|
||||
}
|
||||
if (NullHandling.replaceWithDefault()) {
|
||||
assertFilterMatchesSkipVectorize(edf("dim4 == ''"), ImmutableList.of("1", "2", "6", "7", "8"));
|
||||
} else {
|
||||
|
@ -248,6 +252,10 @@ public class ExpressionFilterTest extends BaseFilterTest
|
|||
assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
}
|
||||
|
||||
// auto type columns don't support mvds, bail out
|
||||
if (testName.contains("AutoTypes")) {
|
||||
return;
|
||||
}
|
||||
// String vs. multi-value string
|
||||
assertFilterMatchesSkipVectorize(edf("dim0 == dim4"), ImmutableList.of("3", "4", "5"));
|
||||
}
|
||||
|
|
|
@ -188,7 +188,7 @@ public class JoinTestHelper
|
|||
|
||||
public static IndexBuilder createFactIndexBuilder(final File tmpDir) throws IOException
|
||||
{
|
||||
return createFactIndexBuilder(TestHelper.NO_CACHE_COLUMN_CONFIG, tmpDir, -1);
|
||||
return createFactIndexBuilder(TestHelper.NO_CACHE_ALWAYS_USE_INDEXES_COLUMN_CONFIG, tmpDir, -1);
|
||||
}
|
||||
|
||||
public static IndexBuilder createFactIndexBuilder(
|
||||
|
|
|
@ -252,6 +252,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
NestedCommonFormatColumnPartSerde partSerde = NestedCommonFormatColumnPartSerde.createDeserializer(
|
||||
ColumnType.NESTED_DATA,
|
||||
false,
|
||||
false,
|
||||
ByteOrder.nativeOrder(),
|
||||
RoaringBitmapSerdeFactory.getInstance()
|
||||
);
|
||||
|
@ -275,6 +276,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
NestedCommonFormatColumnPartSerde partSerde = NestedCommonFormatColumnPartSerde.createDeserializer(
|
||||
ColumnType.NESTED_DATA,
|
||||
false,
|
||||
false,
|
||||
ByteOrder.nativeOrder(),
|
||||
RoaringBitmapSerdeFactory.getInstance()
|
||||
);
|
||||
|
@ -733,13 +735,15 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertNull(dimSelector.getObject());
|
||||
Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0)));
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(rowNumber));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
|
||||
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(rowNumber));
|
||||
|
|
|
@ -515,12 +515,14 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest
|
|||
Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0)));
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(rowNumber));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
|
||||
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(rowNumber));
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.util.concurrent.Futures;
|
|||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
|
@ -46,6 +47,8 @@ import org.apache.druid.segment.column.NullValueIndex;
|
|||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.vector.NoFilterVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -235,7 +238,9 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest
|
|||
private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDoubleColumn column)
|
||||
{
|
||||
SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size());
|
||||
NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(1, 0, data.size());
|
||||
ColumnValueSelector<?> valueSelector = column.makeColumnValueSelector(offset);
|
||||
VectorValueSelector vectorValueSelector = column.makeVectorValueSelector(vectorOffset);
|
||||
|
||||
StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class);
|
||||
DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class);
|
||||
|
@ -255,8 +260,17 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest
|
|||
|
||||
if (row != null) {
|
||||
Assert.assertEquals(row, valueSelector.getObject());
|
||||
Assert.assertEquals((double) row, valueSelector.getDouble(), 0.0);
|
||||
Assert.assertEquals(row, valueSelector.getDouble(), 0.0);
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(row, vectorValueSelector.getDoubleVector()[0], 0.0);
|
||||
Assert.assertEquals(row.longValue(), vectorValueSelector.getLongVector()[0]);
|
||||
Assert.assertEquals(row.floatValue(), vectorValueSelector.getFloatVector()[0], 0.0);
|
||||
boolean[] nullVector = vectorValueSelector.getNullVector();
|
||||
if (NullHandling.sqlCompatible() && nullVector != null) {
|
||||
Assert.assertFalse(nullVector[0]);
|
||||
} else {
|
||||
Assert.assertNull(nullVector);
|
||||
}
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row))))
|
||||
|
@ -275,15 +289,36 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
|
||||
} else {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertTrue(vectorValueSelector.getNullVector()[0]);
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals(NullHandling.defaultDoubleValue(), vectorValueSelector.getDoubleVector()[0], 0.0);
|
||||
Assert.assertNull(vectorValueSelector.getNullVector());
|
||||
|
||||
Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
final String defaultString = String.valueOf(NullHandling.defaultDoubleValue());
|
||||
Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(defaultString))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
}
|
||||
|
||||
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
|
||||
.computeBitmapResult(resultFactory)
|
||||
|
@ -291,6 +326,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
offset.increment();
|
||||
vectorOffset.advance();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.util.concurrent.Futures;
|
|||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
|
@ -46,6 +47,8 @@ import org.apache.druid.segment.column.NullValueIndex;
|
|||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.vector.NoFilterVectorOffset;
|
||||
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -235,7 +238,9 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest
|
|||
private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColumn column)
|
||||
{
|
||||
SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size());
|
||||
NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(1, 0, data.size());
|
||||
ColumnValueSelector<?> valueSelector = column.makeColumnValueSelector(offset);
|
||||
VectorValueSelector vectorValueSelector = column.makeVectorValueSelector(vectorOffset);
|
||||
|
||||
StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class);
|
||||
DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class);
|
||||
|
@ -257,6 +262,15 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(row, valueSelector.getObject());
|
||||
Assert.assertEquals((long) row, valueSelector.getLong());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertEquals((long) row, vectorValueSelector.getLongVector()[0]);
|
||||
Assert.assertEquals(row.doubleValue(), vectorValueSelector.getDoubleVector()[0], 0.0);
|
||||
Assert.assertEquals(row.floatValue(), vectorValueSelector.getFloatVector()[0], 0.0);
|
||||
boolean[] nullVector = vectorValueSelector.getNullVector();
|
||||
if (NullHandling.sqlCompatible() && nullVector != null) {
|
||||
Assert.assertFalse(nullVector[0]);
|
||||
} else {
|
||||
Assert.assertNull(nullVector);
|
||||
}
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row))))
|
||||
|
@ -275,15 +289,32 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
|
||||
} else {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertTrue(valueSelector.isNull());
|
||||
Assert.assertTrue(vectorValueSelector.getNullVector()[0]);
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
} else {
|
||||
Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject());
|
||||
Assert.assertFalse(valueSelector.isNull());
|
||||
Assert.assertNull(vectorValueSelector.getNullVector());
|
||||
Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
final String defaultString = String.valueOf(NullHandling.defaultLongValue());
|
||||
Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(defaultString))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
}
|
||||
|
||||
Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null))
|
||||
.computeBitmapResult(resultFactory)
|
||||
.get(i));
|
||||
Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i));
|
||||
Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH))
|
||||
.computeBitmapResult(resultFactory)
|
||||
|
@ -291,6 +322,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest
|
|||
}
|
||||
|
||||
offset.increment();
|
||||
vectorOffset.advance();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.apache.druid.segment.nested;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -30,10 +31,13 @@ import org.apache.druid.java.util.common.io.Closer;
|
|||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter;
|
||||
import org.apache.druid.math.expr.ExprEval;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.query.DefaultBitmapResultFactory;
|
||||
import org.apache.druid.segment.AutoTypeColumnIndexer;
|
||||
import org.apache.druid.segment.AutoTypeColumnMerger;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.DimensionSelector;
|
||||
import org.apache.druid.segment.IndexSpec;
|
||||
import org.apache.druid.segment.IndexableAdapter;
|
||||
import org.apache.druid.segment.SimpleAscendingOffset;
|
||||
|
@ -41,9 +45,15 @@ import org.apache.druid.segment.column.ColumnBuilder;
|
|||
import org.apache.druid.segment.column.ColumnType;
|
||||
import org.apache.druid.segment.column.DruidPredicateIndex;
|
||||
import org.apache.druid.segment.column.NullValueIndex;
|
||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
||||
import org.apache.druid.segment.column.StringValueSetIndex;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.CompressionFactory;
|
||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
||||
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import org.apache.druid.segment.vector.NoFilterVectorOffset;
|
||||
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
|
@ -54,6 +64,8 @@ import org.junit.BeforeClass;
|
|||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
@ -70,16 +82,15 @@ import java.util.concurrent.CountDownLatch;
|
|||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
||||
@RunWith(Parameterized.class)
|
||||
public class VariantColumnSupplierTest extends InitializedNullHandlingTest
|
||||
{
|
||||
|
||||
@Rule
|
||||
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||
|
||||
BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance();
|
||||
DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapSerdeFactory.getBitmapFactory());
|
||||
|
||||
List<List<Long>> data = Arrays.asList(
|
||||
static List<List<Long>> LONG_ARRAY = Arrays.asList(
|
||||
Collections.emptyList(),
|
||||
Arrays.asList(1L, null, 2L),
|
||||
null,
|
||||
|
@ -88,11 +99,62 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Arrays.asList(null, null)
|
||||
);
|
||||
|
||||
Closer closer = Closer.create();
|
||||
static List<List<Double>> DOUBLE_ARRAY = Arrays.asList(
|
||||
Collections.emptyList(),
|
||||
Arrays.asList(1.1, null, 2.2),
|
||||
null,
|
||||
Collections.singletonList(null),
|
||||
Arrays.asList(3.3, 4.4),
|
||||
Arrays.asList(null, null)
|
||||
);
|
||||
|
||||
SmooshedFileMapper fileMapper;
|
||||
static List<List<String>> STRING_ARRAY = Arrays.asList(
|
||||
Collections.emptyList(),
|
||||
Arrays.asList("a", null, "b"),
|
||||
null,
|
||||
Collections.singletonList(null),
|
||||
Arrays.asList("c", "d"),
|
||||
Arrays.asList(null, null)
|
||||
);
|
||||
|
||||
static List<Object> VARIANT_NUMERIC = Arrays.asList(
|
||||
1L,
|
||||
2.2,
|
||||
null,
|
||||
3.3,
|
||||
4L,
|
||||
null
|
||||
);
|
||||
|
||||
static List<Object> VARIANT_SCALAR = Arrays.asList(
|
||||
null,
|
||||
1L,
|
||||
null,
|
||||
"b",
|
||||
3.3,
|
||||
4L
|
||||
);
|
||||
|
||||
static List<Object> VARIANT_SCALAR_AND_ARRAY = Arrays.asList(
|
||||
Collections.emptyList(),
|
||||
2L,
|
||||
null,
|
||||
Collections.singletonList(null),
|
||||
Arrays.asList(3L, 4L),
|
||||
Arrays.asList(null, "a"),
|
||||
5.5,
|
||||
"b"
|
||||
);
|
||||
|
||||
static List<List<Object>> VARIANT_ARRAY = Arrays.asList(
|
||||
Collections.emptyList(),
|
||||
Arrays.asList("a", null, "b"),
|
||||
null,
|
||||
Collections.singletonList(null),
|
||||
Arrays.asList(3L, 4L),
|
||||
Arrays.asList(null, 3.3)
|
||||
);
|
||||
|
||||
ByteBuffer baseBuffer;
|
||||
|
||||
@BeforeClass
|
||||
public static void staticSetup()
|
||||
|
@ -100,29 +162,74 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
NestedDataModule.registerHandlersAndSerde();
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "data = {0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
IndexSpec fancy = IndexSpec.builder()
|
||||
.withLongEncoding(CompressionFactory.LongEncodingStrategy.AUTO)
|
||||
.withStringDictionaryEncoding(
|
||||
new StringEncodingStrategy.FrontCoded(16, FrontCodedIndexed.V1)
|
||||
)
|
||||
.build();
|
||||
final List<Object[]> constructors = ImmutableList.of(
|
||||
new Object[]{"ARRAY<LONG>", LONG_ARRAY, IndexSpec.DEFAULT},
|
||||
new Object[]{"ARRAY<LONG>", LONG_ARRAY, fancy},
|
||||
new Object[]{"ARRAY<DOUBLE>", DOUBLE_ARRAY, IndexSpec.DEFAULT},
|
||||
new Object[]{"ARRAY<DOUBLE>", DOUBLE_ARRAY, fancy},
|
||||
new Object[]{"ARRAY<STRING>", STRING_ARRAY, IndexSpec.DEFAULT},
|
||||
new Object[]{"ARRAY<STRING>", STRING_ARRAY, fancy},
|
||||
new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, IndexSpec.DEFAULT},
|
||||
new Object[]{"DOUBLE,LONG", VARIANT_NUMERIC, fancy},
|
||||
new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, IndexSpec.DEFAULT},
|
||||
new Object[]{"DOUBLE,LONG,STRING", VARIANT_SCALAR, fancy},
|
||||
new Object[]{"ARRAY<LONG>,ARRAY<STRING>,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, IndexSpec.DEFAULT},
|
||||
new Object[]{"ARRAY<LONG>,ARRAY<STRING>,DOUBLE,LONG,STRING", VARIANT_SCALAR_AND_ARRAY, fancy},
|
||||
new Object[]{"ARRAY<DOUBLE>,ARRAY<LONG>,ARRAY<STRING>", VARIANT_ARRAY, IndexSpec.DEFAULT},
|
||||
new Object[]{"ARRAY<DOUBLE>,ARRAY<LONG>,ARRAY<STRING>", VARIANT_ARRAY, fancy}
|
||||
);
|
||||
|
||||
return constructors;
|
||||
}
|
||||
|
||||
Closer closer = Closer.create();
|
||||
|
||||
SmooshedFileMapper fileMapper;
|
||||
|
||||
ByteBuffer baseBuffer;
|
||||
|
||||
FieldTypeInfo.MutableTypeSet expectedTypes;
|
||||
|
||||
ColumnType expectedLogicalType = null;
|
||||
|
||||
private final List<?> data;
|
||||
private final IndexSpec indexSpec;
|
||||
|
||||
public VariantColumnSupplierTest(
|
||||
@SuppressWarnings("unused") String name,
|
||||
List<?> data,
|
||||
IndexSpec indexSpec
|
||||
)
|
||||
{
|
||||
this.data = data;
|
||||
this.indexSpec = indexSpec;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException
|
||||
{
|
||||
final String fileNameBase = "test";
|
||||
fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data);
|
||||
fileMapper = smooshify(fileNameBase, tempFolder.newFolder());
|
||||
baseBuffer = fileMapper.mapFile(fileNameBase);
|
||||
}
|
||||
|
||||
private SmooshedFileMapper smooshify(
|
||||
String fileNameBase,
|
||||
File tmpFile,
|
||||
List<?> data
|
||||
File tmpFile
|
||||
)
|
||||
throws IOException
|
||||
{
|
||||
SegmentWriteOutMediumFactory writeOutMediumFactory = TmpFileSegmentWriteOutMediumFactory.instance();
|
||||
try (final FileSmoosher smoosher = new FileSmoosher(tmpFile)) {
|
||||
VariantArrayColumnSerializer serializer = new VariantArrayColumnSerializer(
|
||||
fileNameBase,
|
||||
IndexSpec.DEFAULT,
|
||||
writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()),
|
||||
closer
|
||||
);
|
||||
|
||||
AutoTypeColumnIndexer indexer = new AutoTypeColumnIndexer();
|
||||
for (Object o : data) {
|
||||
|
@ -136,6 +243,18 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
SortedValueDictionary globalDictionarySortedCollector = mergable.getValueDictionary();
|
||||
mergable.mergeFieldsInto(sortedFields);
|
||||
|
||||
expectedTypes = sortedFields.get(NestedPathFinder.JSON_PATH_ROOT);
|
||||
for (ColumnType type : FieldTypeInfo.convertToSet(expectedTypes.getByteValue())) {
|
||||
expectedLogicalType = ColumnType.leastRestrictiveType(expectedLogicalType, type);
|
||||
}
|
||||
VariantColumnSerializer serializer = new VariantColumnSerializer(
|
||||
fileNameBase,
|
||||
expectedTypes.getSingleType() == null ? expectedTypes.getByteValue() : null,
|
||||
indexSpec,
|
||||
writeOutMediumFactory.makeSegmentWriteOutMedium(tempFolder.newFolder()),
|
||||
closer
|
||||
);
|
||||
|
||||
serializer.openDictionaryWriter();
|
||||
serializer.serializeDictionaries(
|
||||
globalDictionarySortedCollector.getSortedStrings(),
|
||||
|
@ -173,16 +292,16 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
{
|
||||
ColumnBuilder bob = new ColumnBuilder();
|
||||
bob.setFileMapper(fileMapper);
|
||||
VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read(
|
||||
ColumnType.LONG_ARRAY,
|
||||
VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
|
||||
expectedLogicalType,
|
||||
ByteOrder.nativeOrder(),
|
||||
bitmapSerdeFactory,
|
||||
baseBuffer,
|
||||
bob,
|
||||
NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES
|
||||
);
|
||||
try (VariantArrayColumn column = (VariantArrayColumn) supplier.get()) {
|
||||
smokeTest(supplier, column);
|
||||
try (VariantColumn<?> column = (VariantColumn<?>) supplier.get()) {
|
||||
smokeTest(supplier, column, data, expectedTypes);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -192,8 +311,8 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
// if this test ever starts being to be a flake, there might be thread safety issues
|
||||
ColumnBuilder bob = new ColumnBuilder();
|
||||
bob.setFileMapper(fileMapper);
|
||||
VariantArrayColumnAndIndexSupplier supplier = VariantArrayColumnAndIndexSupplier.read(
|
||||
ColumnType.LONG_ARRAY,
|
||||
VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read(
|
||||
expectedLogicalType,
|
||||
ByteOrder.nativeOrder(),
|
||||
bitmapSerdeFactory,
|
||||
baseBuffer,
|
||||
|
@ -215,8 +334,8 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
try {
|
||||
threadsStartLatch.await();
|
||||
for (int iter = 0; iter < 5000; iter++) {
|
||||
try (VariantArrayColumn column = (VariantArrayColumn) supplier.get()) {
|
||||
smokeTest(supplier, column);
|
||||
try (VariantColumn column = (VariantColumn) supplier.get()) {
|
||||
smokeTest(supplier, column, data, expectedTypes);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -231,10 +350,21 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
Assert.assertEquals(expectedReason, failureReason.get());
|
||||
}
|
||||
|
||||
private void smokeTest(VariantArrayColumnAndIndexSupplier supplier, VariantArrayColumn column)
|
||||
private void smokeTest(
|
||||
VariantColumnAndIndexSupplier supplier,
|
||||
VariantColumn<?> column,
|
||||
List<?> data,
|
||||
FieldTypeInfo.MutableTypeSet expectedType
|
||||
)
|
||||
{
|
||||
SimpleAscendingOffset offset = new SimpleAscendingOffset(data.size());
|
||||
NoFilterVectorOffset vectorOffset = new NoFilterVectorOffset(1, 0, data.size());
|
||||
ColumnValueSelector<?> valueSelector = column.makeColumnValueSelector(offset);
|
||||
DimensionSelector dimensionSelector =
|
||||
expectedLogicalType.isPrimitive() ? column.makeDimensionSelector(offset, null) : null;
|
||||
VectorObjectSelector vectorObjectSelector = column.makeVectorObjectSelector(vectorOffset);
|
||||
SingleValueDimensionVectorSelector dimensionVectorSelector =
|
||||
expectedLogicalType.isPrimitive() ? column.makeSingleValueDimensionVectorSelector(vectorOffset) : null;
|
||||
|
||||
StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class);
|
||||
Assert.assertNull(valueSetIndex);
|
||||
|
@ -245,26 +375,69 @@ public class VariantArrayColumnSupplierTest extends InitializedNullHandlingTest
|
|||
|
||||
SortedMap<String, FieldTypeInfo.MutableTypeSet> fields = column.getFieldTypeInfo();
|
||||
Assert.assertEquals(
|
||||
ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, new FieldTypeInfo.MutableTypeSet().add(ColumnType.LONG_ARRAY)),
|
||||
ImmutableMap.of(NestedPathFinder.JSON_PATH_ROOT, expectedType),
|
||||
fields
|
||||
);
|
||||
final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(expectedLogicalType);
|
||||
|
||||
for (int i = 0; i < data.size(); i++) {
|
||||
List<Long> row = data.get(i);
|
||||
Object row = data.get(i);
|
||||
|
||||
// in default value mode, even though the input row had an empty string, the selector spits out null, so we want
|
||||
// to take the null checking path
|
||||
|
||||
if (row != null) {
|
||||
Assert.assertArrayEquals(row.toArray(), (Object[]) valueSelector.getObject());
|
||||
if (row instanceof List) {
|
||||
Assert.assertArrayEquals(((List) row).toArray(), (Object[]) valueSelector.getObject());
|
||||
if (expectedType.getSingleType() != null) {
|
||||
Assert.assertArrayEquals(((List) row).toArray(), (Object[]) vectorObjectSelector.getObjectVector()[0]);
|
||||
} else {
|
||||
// mixed type vector object selector coerces to the most common type
|
||||
Assert.assertArrayEquals(ExprEval.ofType(expressionType, row).asArray(), (Object[]) vectorObjectSelector.getObjectVector()[0]);
|
||||
}
|
||||
} else {
|
||||
Assert.assertEquals(row, valueSelector.getObject());
|
||||
if (expectedType.getSingleType() != null) {
|
||||
Assert.assertEquals(
|
||||
row,
|
||||
vectorObjectSelector.getObjectVector()[0]
|
||||
);
|
||||
} else {
|
||||
// vector object selector always coerces to the most common type
|
||||
ExprEval eval = ExprEval.ofType(expressionType, row);
|
||||
if (expectedLogicalType.isArray()) {
|
||||
Assert.assertArrayEquals(eval.asArray(), (Object[]) vectorObjectSelector.getObjectVector()[0]);
|
||||
} else {
|
||||
Assert.assertEquals(eval.value(), vectorObjectSelector.getObjectVector()[0]);
|
||||
}
|
||||
}
|
||||
if (dimensionSelector != null) {
|
||||
Assert.assertEquals(String.valueOf(row), dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
// null is always 0
|
||||
Assert.assertTrue(dimensionSelector.idLookup().lookupId(String.valueOf(row)) > 0);
|
||||
if (dimensionVectorSelector != null) {
|
||||
int[] dim = dimensionVectorSelector.getRowVector();
|
||||
Assert.assertEquals(String.valueOf(row), dimensionVectorSelector.lookupName(dim[0]));
|
||||
}
|
||||
}
|
||||
}
|
||||
Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
|
||||
} else {
|
||||
Assert.assertNull(valueSelector.getObject());
|
||||
Assert.assertNull(vectorObjectSelector.getObjectVector()[0]);
|
||||
if (dimensionSelector != null) {
|
||||
Assert.assertNull(dimensionSelector.lookupName(dimensionSelector.getRow().get(0)));
|
||||
Assert.assertEquals(0, dimensionSelector.idLookup().lookupId(null));
|
||||
if (dimensionVectorSelector != null) {
|
||||
Assert.assertNull(dimensionVectorSelector.lookupName(dimensionVectorSelector.getRowVector()[0]));
|
||||
}
|
||||
}
|
||||
Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i));
|
||||
}
|
||||
|
||||
offset.increment();
|
||||
vectorOffset.advance();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.math.expr.ExpressionType;
|
||||
import org.apache.druid.math.expr.Parser;
|
||||
import org.apache.druid.query.expression.NestedDataExpressions;
|
||||
import org.apache.druid.query.expression.TestExprMacroTable;
|
||||
import org.apache.druid.segment.ColumnInspector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
|
@ -824,7 +823,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest
|
|||
ExpressionPlan.Trait.VECTORIZABLE
|
||||
)
|
||||
);
|
||||
Assert.assertEquals(NestedDataExpressions.TYPE, thePlan.getOutputType());
|
||||
Assert.assertEquals(ExpressionType.NESTED_DATA, thePlan.getOutputType());
|
||||
ColumnCapabilities inferred = thePlan.inferColumnCapabilities(
|
||||
ExpressionType.toColumnType(thePlan.getOutputType())
|
||||
);
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.google.inject.Key;
|
|||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.guice.NestedDataModule;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -168,7 +169,17 @@ public class DumpSegmentTest extends InitializedNullHandlingTest
|
|||
);
|
||||
final byte[] fileBytes = Files.readAllBytes(outputFile.toPath());
|
||||
final String output = StringUtils.fromUtf8(fileBytes);
|
||||
Assert.assertEquals("{\"nest\":{\"fields\":[{\"path\":\"$.x\",\"types\":[\"LONG\"]},{\"path\":\"$.y\",\"types\":[\"DOUBLE\"]},{\"path\":\"$.z\",\"types\":[\"STRING\"]}],\"dictionaries\":{\"strings\":[{\"globalId\":0,\"value\":null},{\"globalId\":1,\"value\":\"a\"},{\"globalId\":2,\"value\":\"b\"}],\"longs\":[{\"globalId\":3,\"value\":100},{\"globalId\":4,\"value\":200},{\"globalId\":5,\"value\":400}],\"doubles\":[{\"globalId\":6,\"value\":1.1},{\"globalId\":7,\"value\":2.2},{\"globalId\":8,\"value\":3.3}],\"nullRows\":[]}}}", output);
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertEquals(
|
||||
"{\"nest\":{\"fields\":[{\"path\":\"$.x\",\"types\":[\"LONG\"]},{\"path\":\"$.y\",\"types\":[\"DOUBLE\"]},{\"path\":\"$.z\",\"types\":[\"STRING\"]}],\"dictionaries\":{\"strings\":[{\"globalId\":0,\"value\":null},{\"globalId\":1,\"value\":\"a\"},{\"globalId\":2,\"value\":\"b\"}],\"longs\":[{\"globalId\":3,\"value\":100},{\"globalId\":4,\"value\":200},{\"globalId\":5,\"value\":400}],\"doubles\":[{\"globalId\":6,\"value\":1.1},{\"globalId\":7,\"value\":2.2},{\"globalId\":8,\"value\":3.3}],\"nullRows\":[]}}}",
|
||||
output
|
||||
);
|
||||
} else {
|
||||
Assert.assertEquals(
|
||||
"{\"nest\":{\"fields\":[{\"path\":\"$.x\",\"types\":[\"LONG\"]},{\"path\":\"$.y\",\"types\":[\"DOUBLE\"]},{\"path\":\"$.z\",\"types\":[\"STRING\"]}],\"dictionaries\":{\"strings\":[{\"globalId\":0,\"value\":null},{\"globalId\":1,\"value\":\"a\"},{\"globalId\":2,\"value\":\"b\"}],\"longs\":[{\"globalId\":3,\"value\":0},{\"globalId\":4,\"value\":100},{\"globalId\":5,\"value\":200},{\"globalId\":6,\"value\":400}],\"doubles\":[{\"globalId\":7,\"value\":0.0},{\"globalId\":8,\"value\":1.1},{\"globalId\":9,\"value\":2.2},{\"globalId\":10,\"value\":3.3}],\"nullRows\":[]}}}",
|
||||
output
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -191,7 +202,17 @@ public class DumpSegmentTest extends InitializedNullHandlingTest
|
|||
);
|
||||
final byte[] fileBytes = Files.readAllBytes(outputFile.toPath());
|
||||
final String output = StringUtils.fromUtf8(fileBytes);
|
||||
Assert.assertEquals("{\"bitmapSerdeFactory\":{\"type\":\"roaring\"},\"nest\":{\"$.x\":{\"types\":[\"LONG\"],\"dictionary\":[{\"localId\":0,\"globalId\":0,\"value\":null,\"rows\":[4]},{\"localId\":1,\"globalId\":3,\"value\":\"100\",\"rows\":[3]},{\"localId\":2,\"globalId\":4,\"value\":\"200\",\"rows\":[0,2]},{\"localId\":3,\"globalId\":5,\"value\":\"400\",\"rows\":[1]}],\"column\":[{\"row\":0,\"raw\":{\"x\":200,\"y\":2.2},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":1,\"raw\":{\"x\":400,\"y\":1.1,\"z\":\"a\"},\"fieldId\":3,\"fieldValue\":\"400\"},{\"row\":2,\"raw\":{\"x\":200,\"z\":\"b\"},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":3,\"raw\":{\"x\":100,\"y\":1.1,\"z\":\"a\"},\"fieldId\":1,\"fieldValue\":\"100\"},{\"row\":4,\"raw\":{\"y\":3.3,\"z\":\"b\"},\"fieldId\":0,\"fieldValue\":null}]}}}", output);
|
||||
if (NullHandling.sqlCompatible()) {
|
||||
Assert.assertEquals(
|
||||
"{\"bitmapSerdeFactory\":{\"type\":\"roaring\"},\"nest\":{\"$.x\":{\"types\":[\"LONG\"],\"dictionary\":[{\"localId\":0,\"globalId\":0,\"value\":null,\"rows\":[4]},{\"localId\":1,\"globalId\":3,\"value\":\"100\",\"rows\":[3]},{\"localId\":2,\"globalId\":4,\"value\":\"200\",\"rows\":[0,2]},{\"localId\":3,\"globalId\":5,\"value\":\"400\",\"rows\":[1]}],\"column\":[{\"row\":0,\"raw\":{\"x\":200,\"y\":2.2},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":1,\"raw\":{\"x\":400,\"y\":1.1,\"z\":\"a\"},\"fieldId\":3,\"fieldValue\":\"400\"},{\"row\":2,\"raw\":{\"x\":200,\"z\":\"b\"},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":3,\"raw\":{\"x\":100,\"y\":1.1,\"z\":\"a\"},\"fieldId\":1,\"fieldValue\":\"100\"},{\"row\":4,\"raw\":{\"y\":3.3,\"z\":\"b\"},\"fieldId\":0,\"fieldValue\":null}]}}}",
|
||||
output
|
||||
);
|
||||
} else {
|
||||
Assert.assertEquals(
|
||||
"{\"bitmapSerdeFactory\":{\"type\":\"roaring\"},\"nest\":{\"$.x\":{\"types\":[\"LONG\"],\"dictionary\":[{\"localId\":0,\"globalId\":0,\"value\":null,\"rows\":[4]},{\"localId\":1,\"globalId\":4,\"value\":\"100\",\"rows\":[3]},{\"localId\":2,\"globalId\":5,\"value\":\"200\",\"rows\":[0,2]},{\"localId\":3,\"globalId\":6,\"value\":\"400\",\"rows\":[1]}],\"column\":[{\"row\":0,\"raw\":{\"x\":200,\"y\":2.2},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":1,\"raw\":{\"x\":400,\"y\":1.1,\"z\":\"a\"},\"fieldId\":3,\"fieldValue\":\"400\"},{\"row\":2,\"raw\":{\"x\":200,\"z\":\"b\"},\"fieldId\":2,\"fieldValue\":\"200\"},{\"row\":3,\"raw\":{\"x\":100,\"y\":1.1,\"z\":\"a\"},\"fieldId\":1,\"fieldValue\":\"100\"},{\"row\":4,\"raw\":{\"y\":3.3,\"z\":\"b\"},\"fieldId\":0,\"fieldValue\":null}]}}}",
|
||||
output
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
public static List<Segment> createSegments(
|
||||
|
|
|
@ -437,6 +437,22 @@ public class DruidExpression
|
|||
);
|
||||
}
|
||||
|
||||
public DruidExpression map(
|
||||
final Function<SimpleExtraction, SimpleExtraction> extractionMap,
|
||||
final Function<String, String> expressionMap,
|
||||
final ColumnType newType
|
||||
)
|
||||
{
|
||||
return new DruidExpression(
|
||||
nodeType,
|
||||
newType,
|
||||
simpleExtraction == null ? null : extractionMap.apply(simpleExtraction),
|
||||
(args) -> expressionMap.apply(expressionGenerator.compile(args)),
|
||||
arguments,
|
||||
virtualColumnCreator
|
||||
);
|
||||
}
|
||||
|
||||
public DruidExpression withArguments(List<DruidExpression> newArgs)
|
||||
{
|
||||
return new DruidExpression(
|
||||
|
|
|
@ -508,7 +508,15 @@ public class Expressions
|
|||
}
|
||||
|
||||
final DimFilter equalFilter;
|
||||
if (druidExpression.isSimpleExtraction()) {
|
||||
final ColumnType outputType = druidExpression.getDruidType();
|
||||
final boolean isOutputNumeric = outputType != null && outputType.isNumeric();
|
||||
// if a simple extraction, we can typically use the base column directly for filtering. however, some expressions
|
||||
// such as cast also appear as a simple extraction because some native layer things can handle the cast
|
||||
// themselves, so we check the output type of the expression and compare it to the type of the direct column. a
|
||||
// string column might produce additional null values when converting to a number, so we should use the virtual
|
||||
// column instead for filtering to ensure that results are correct
|
||||
if (druidExpression.isSimpleExtraction() &&
|
||||
!(isOutputNumeric && !rowSignature.isNumeric(druidExpression.getDirectColumn()))) {
|
||||
equalFilter = new SelectorDimFilter(
|
||||
druidExpression.getSimpleExtraction().getColumn(),
|
||||
NullHandling.defaultStringValue(),
|
||||
|
|
|
@ -39,8 +39,8 @@ public class ArrayAppendOperatorConversion extends DirectOperatorConversion
|
|||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
),
|
||||
OperandTypes.or(
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
OperandTypes.family(SqlTypeFamily.STRING),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -37,7 +37,7 @@ public class ArrayOffsetOperatorConversion extends DirectOperatorConversion
|
|||
OperandTypes.family(SqlTypeFamily.ARRAY),
|
||||
OperandTypes.family(SqlTypeFamily.STRING)
|
||||
),
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
OperandTypes.family(SqlTypeFamily.NUMERIC)
|
||||
)
|
||||
)
|
||||
.functionCategory(SqlFunctionCategory.STRING)
|
||||
|
|
|
@ -108,7 +108,8 @@ public class CastOperatorConversion implements SqlOperatorConversion
|
|||
// Ignore casts for simple extractions (use Function.identity) since it is ok in many cases.
|
||||
typeCastExpression = operandExpression.map(
|
||||
Function.identity(),
|
||||
expression -> StringUtils.format("CAST(%s, '%s')", expression, toExpressionType.asTypeString())
|
||||
expression -> StringUtils.format("CAST(%s, '%s')", expression, toExpressionType.asTypeString()),
|
||||
toDruidType
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -109,6 +109,7 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
ImmutableMap.<String, Object>builder()
|
||||
.put("t", "2000-01-01")
|
||||
.put("string", "ccc")
|
||||
.put("string_sparse", "10")
|
||||
.put("nest", ImmutableMap.of("x", 200L, "y", 3.03, "z", "abcdef", "mixed", 1.1, "mixed2", 1L))
|
||||
.put("long", 3L)
|
||||
.build(),
|
||||
|
@ -705,7 +706,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 10L},
|
||||
new Object[]{NullHandling.defaultStringValue(), 8L},
|
||||
new Object[]{"10", 2L},
|
||||
new Object[]{"yyy", 2L},
|
||||
new Object[]{"zzz", 2L}
|
||||
),
|
||||
|
@ -811,7 +813,8 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 10L},
|
||||
new Object[]{NullHandling.defaultStringValue(), 8L},
|
||||
new Object[]{"10", 2L},
|
||||
new Object[]{"yyy", 2L},
|
||||
new Object[]{"zzz", 2L}
|
||||
),
|
||||
|
@ -2705,7 +2708,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 4L},
|
||||
new Object[]{"100", 2L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
|
@ -2967,7 +2969,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 4L},
|
||||
new Object[]{"2.02", 2L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
|
@ -4561,4 +4562,285 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByPathSelectorFilterVariantNull()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(nest, '$.x'), "
|
||||
+ "JSON_VALUE(nester, '$.n.x' RETURNING BIGINT), "
|
||||
+ "SUM(cnt) "
|
||||
+ "FROM druid.nested WHERE JSON_VALUE(nester, '$.n.x' RETURNING BIGINT) IS NULL GROUP BY 1, 2",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(DATA_SOURCE)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn("nester", "$.n.x", "v0", ColumnType.LONG),
|
||||
new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING)
|
||||
)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v1", "d0"),
|
||||
new DefaultDimensionSpec("v0", "d1", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setDimFilter(selector("v0", null, null))
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), NullHandling.defaultLongValue(), 4L},
|
||||
new Object[]{"100", NullHandling.defaultLongValue(), 1L},
|
||||
new Object[]{"200", NullHandling.defaultLongValue(), 1L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.STRING)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.add("EXPR$2", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSelectPathSelectorFilterVariantNull()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(nest, '$.x'), "
|
||||
+ "JSON_VALUE(nester, '$.n.x' RETURNING BIGINT) "
|
||||
+ "FROM druid.nested WHERE JSON_VALUE(nester, '$.n.x' RETURNING BIGINT) IS NULL",
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
new NestedFieldVirtualColumn("nester", "$.n.x", "v0", ColumnType.LONG),
|
||||
new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING)
|
||||
)
|
||||
.columns(
|
||||
"v0", "v1"
|
||||
)
|
||||
.filters(selector("v0", null, null))
|
||||
.context(QUERY_CONTEXT_DEFAULT)
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"100", null},
|
||||
new Object[]{NullHandling.defaultStringValue(), null},
|
||||
new Object[]{"200", null},
|
||||
new Object[]{NullHandling.defaultStringValue(), null},
|
||||
new Object[]{NullHandling.defaultStringValue(), null},
|
||||
new Object[]{NullHandling.defaultStringValue(), null}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.STRING)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByPathSelectorFilterVariantNotNull()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(nest, '$.x'), "
|
||||
+ "JSON_VALUE(nester, '$.n.x' RETURNING BIGINT), "
|
||||
+ "SUM(cnt) "
|
||||
+ "FROM druid.nested WHERE JSON_VALUE(nester, '$.n.x' RETURNING BIGINT) IS NOT NULL GROUP BY 1, 2",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(DATA_SOURCE)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setVirtualColumns(
|
||||
new NestedFieldVirtualColumn("nester", "$.n.x", "v0", ColumnType.LONG),
|
||||
new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING)
|
||||
)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v1", "d0"),
|
||||
new DefaultDimensionSpec("v0", "d1", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setDimFilter(not(selector("v0", null, null)))
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{"100", 1L, 1L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.STRING)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.add("EXPR$2", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRegularLongLongMixed1FilterNotNull()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(long, '$' RETURNING BIGINT), "
|
||||
+ "SUM(cnt) "
|
||||
+ "FROM druid.nested_mix WHERE JSON_VALUE(long, '$' RETURNING BIGINT) IS NOT NULL GROUP BY 1",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(DATA_SOURCE_MIXED)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setVirtualColumns(new NestedFieldVirtualColumn("long", "$", "v0", ColumnType.LONG))
|
||||
.setDimFilter(not(selector("v0", null, null)))
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{1L, 2L},
|
||||
new Object[]{2L, 4L},
|
||||
new Object[]{3L, 2L},
|
||||
new Object[]{4L, 2L},
|
||||
new Object[]{5L, 4L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.LONG)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootSingleTypeStringMixed1SparseNotNull()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "JSON_VALUE(string_sparse, '$' RETURNING BIGINT), "
|
||||
+ "SUM(cnt) "
|
||||
+ "FROM druid.nested_mix_2 WHERE JSON_VALUE(string_sparse, '$' RETURNING BIGINT) IS NOT NULL GROUP BY 1",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(DATA_SOURCE_MIXED_2)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setVirtualColumns(new NestedFieldVirtualColumn("string_sparse", "$", "v0", ColumnType.LONG))
|
||||
.setDimFilter(not(selector("v0", null, null)))
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{10L, 2L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.LONG)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanStringNotNullCast()
|
||||
{
|
||||
skipVectorize();
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "CAST(string_sparse as BIGINT)"
|
||||
+ "FROM druid.nested_mix WHERE CAST(string_sparse as BIGINT) IS NOT NULL",
|
||||
ImmutableList.of(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(DATA_SOURCE_MIXED)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.virtualColumns(
|
||||
expressionVirtualColumn("v0", "CAST(\"string_sparse\", 'LONG')", ColumnType.LONG)
|
||||
)
|
||||
.filters(not(selector("v0", null, null)))
|
||||
.columns("v0")
|
||||
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||
.legacy(false)
|
||||
.build()
|
||||
),
|
||||
NullHandling.sqlCompatible() ?
|
||||
ImmutableList.of(
|
||||
new Object[]{10L},
|
||||
new Object[]{10L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{10L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L},
|
||||
new Object[]{0L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByRootSingleTypeStringMixed1SparseNotNullCast2()
|
||||
{
|
||||
testQuery(
|
||||
"SELECT "
|
||||
+ "CAST(string_sparse as BIGINT), "
|
||||
+ "SUM(cnt) "
|
||||
+ "FROM druid.nested_mix WHERE CAST(string_sparse as BIGINT) IS NOT NULL GROUP BY 1",
|
||||
ImmutableList.of(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(DATA_SOURCE_MIXED)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
dimensions(
|
||||
new DefaultDimensionSpec("string_sparse", "d0", ColumnType.LONG)
|
||||
)
|
||||
)
|
||||
.setVirtualColumns(expressionVirtualColumn("v0", "CAST(\"string_sparse\", 'LONG')", ColumnType.LONG))
|
||||
.setDimFilter(not(selector("v0", null, null)))
|
||||
.setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
|
||||
.setContext(QUERY_CONTEXT_DEFAULT)
|
||||
.build()
|
||||
),
|
||||
NullHandling.sqlCompatible() ?
|
||||
ImmutableList.of(
|
||||
new Object[]{10L, 2L}
|
||||
) :
|
||||
ImmutableList.of(
|
||||
new Object[]{0L, 12L},
|
||||
new Object[]{10L, 2L}
|
||||
),
|
||||
RowSignature.builder()
|
||||
.add("EXPR$0", ColumnType.LONG)
|
||||
.add("EXPR$1", ColumnType.LONG)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue