diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java index 98440fcac949..fcc4cf24794a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -145,7 +146,7 @@ public void timeFloorUsingExpression(Blackhole blackhole) new ExpressionVirtualColumn( "v", "timestamp_floor(__time, 'PT1H')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) @@ -239,7 +240,7 @@ public void timeFormatUsingExpression(Blackhole blackhole) new ExpressionVirtualColumn( "v", "timestamp_format(__time, 'yyyy-MM-dd')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -304,7 +305,7 @@ public void strlenUsingExpressionAsLong(Blackhole blackhole) new ExpressionVirtualColumn( "v", "strlen(s)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -336,7 +337,7 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) new ExpressionVirtualColumn( "v", "strlen(s)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -350,7 +351,7 @@ public void strlenUsingExpressionAsString(Blackhole blackhole) .map(cursor -> { final DimensionSelector selector = cursor .getColumnSelectorFactory() - .makeDimensionSelector(new DefaultDimensionSpec("v", "v", ValueType.STRING)); + .makeDimensionSelector(new DefaultDimensionSpec("v", "v", ColumnType.STRING)); consumeDimension(cursor, selector, blackhole); return null; @@ -397,7 +398,7 @@ public void arithmeticOnLong(Blackhole blackhole) new ExpressionVirtualColumn( "v", "n + 1", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) @@ -429,7 +430,7 @@ public void stringConcatAndCompareOnLong(Blackhole blackhole) new ExpressionVirtualColumn( "v", "concat(n, ' is my favorite number') == '3 is my favorite number'", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java index e969a5f4c93c..a0e06d019fd5 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java @@ -27,6 +27,7 @@ import org.apache.druid.math.expr.Expr; 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.Parser; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.ColumnInspector; @@ -102,7 +103,7 @@ public class ExpressionVectorSelectorBenchmark private Closer closer; @Nullable - private ExprType outputType; + private ExpressionType outputType; @Setup(Level.Trial) public void setup() @@ -155,7 +156,7 @@ public void scan(Blackhole blackhole) new ExpressionVirtualColumn( "v", expression, - ExprType.toValueType(outputType), + ExpressionType.toColumnType(outputType), TestExprMacroTable.INSTANCE ) ) @@ -171,7 +172,7 @@ public void scan(Blackhole blackhole) ); if (outputType.isNumeric()) { VectorValueSelector selector = cursor.getColumnSelectorFactory().makeValueSelector("v"); - if (outputType.equals(ExprType.DOUBLE)) { + if (outputType.is(ExprType.DOUBLE)) { while (!cursor.isDone()) { blackhole.consume(selector.getDoubleVector()); blackhole.consume(selector.getNullVector()); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java index ee6b7e3c8e94..d2fed4b0fab1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -42,7 +42,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.join.HashJoinSegment; import org.apache.druid.segment.join.JoinConditionAnalysis; @@ -319,13 +319,13 @@ public Optional get(String lookupName) new ExpressionVirtualColumn( LOOKUP_COUNTRY_CODE_TO_NAME, "lookup(countryIsoCode, '" + LOOKUP_COUNTRY_CODE_TO_NAME + "')", - ValueType.STRING, + ColumnType.STRING, exprMacroTable ), new ExpressionVirtualColumn( LOOKUP_COUNTRY_NUMBER_TO_NAME, "lookup(countryNumber, '" + LOOKUP_COUNTRY_NUMBER_TO_NAME + "')", - ValueType.STRING, + ColumnType.STRING, exprMacroTable ) ) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java index aa2f119754bd..1fc373f713af 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/ColumnarLongsEncodeDataFromSegmentBenchmark.java @@ -128,9 +128,8 @@ private void initializeSegmentValueIntermediaryFile() throws IOException Iterables.addAll(columnNames, index.getColumnNames()); final ColumnHolder column = index.getColumnHolder(columnName); final ColumnCapabilities capabilities = column.getCapabilities(); - final ValueType columnType = capabilities.getType(); try (Writer writer = Files.newBufferedWriter(dataFile.toPath(), StandardCharsets.UTF_8)) { - if (columnType != ValueType.LONG) { + if (!capabilities.is(ValueType.LONG)) { throw new RuntimeException("Invalid column type, expected 'Long'"); } LongsColumn theColumn = (LongsColumn) column.getColumn(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index b9808c9cad69..bbf98fd7ff54 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -81,7 +81,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.generator.DataGenerator; import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; @@ -333,7 +333,7 @@ private void setupQueries() .builder() .setDataSource("blah") .setQuerySegmentSpec(intervalSpec) - .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ColumnType.STRING)) .setAggregatorSpecs( queryAggs ) @@ -359,7 +359,7 @@ private void setupQueries() .builder() .setDataSource("blah") .setQuerySegmentSpec(intervalSpec) - .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ColumnType.LONG)) .setAggregatorSpecs( queryAggs ) @@ -385,7 +385,7 @@ private void setupQueries() .builder() .setDataSource("blah") .setQuerySegmentSpec(intervalSpec) - .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT)) + .setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ColumnType.FLOAT)) .setAggregatorSpecs(queryAggs) .setGranularity(Granularity.fromString(queryGranularity)) .setContext(ImmutableMap.of("vectorize", vectorize)) @@ -411,7 +411,7 @@ private void setupQueries() .builder() .setDataSource("blah") .setQuerySegmentSpec(intervalSpec) - .setDimensions(new DefaultDimensionSpec("stringZipf", "stringZipf", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("stringZipf", "stringZipf", ColumnType.STRING)) .setAggregatorSpecs( queryAggs ) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java index 263f619a1dbe..c57f4728c403 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DimensionSchema.java @@ -29,7 +29,7 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Objects; @@ -126,7 +126,7 @@ public boolean hasBitmapIndex() public abstract String getTypeName(); @JsonIgnore - public abstract ValueType getValueType(); + public abstract ColumnType getColumnType(); @Override public boolean equals(final Object o) @@ -141,14 +141,14 @@ public boolean equals(final Object o) return createBitmapIndex == that.createBitmapIndex && Objects.equals(name, that.name) && Objects.equals(getTypeName(), that.getTypeName()) && - Objects.equals(getValueType(), that.getValueType()) && + Objects.equals(getColumnType(), that.getColumnType()) && multiValueHandling == that.multiValueHandling; } @Override public int hashCode() { - return Objects.hash(name, multiValueHandling, createBitmapIndex, getTypeName(), getValueType()); + return Objects.hash(name, multiValueHandling, createBitmapIndex, getTypeName(), getColumnType()); } @Override @@ -156,7 +156,7 @@ public String toString() { return "DimensionSchema{" + "name='" + name + '\'' + - ", valueType=" + getValueType() + + ", valueType=" + getColumnType() + ", typeName=" + getTypeName() + ", multiValueHandling=" + multiValueHandling + ", createBitmapIndex=" + createBitmapIndex + diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java index 7e45d1e89aab..01c42d031529 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DoubleDimensionSchema.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; public class DoubleDimensionSchema extends DimensionSchema { @@ -38,8 +38,8 @@ public String getTypeName() } @Override - public ValueType getValueType() + public ColumnType getColumnType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java index a9a3ad66770a..9514107a1728 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FloatDimensionSchema.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; public class FloatDimensionSchema extends DimensionSchema { @@ -42,8 +42,8 @@ public String getTypeName() @Override @JsonIgnore - public ValueType getValueType() + public ColumnType getColumnType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java index 6af9d4ebfb50..a9533f279906 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LongDimensionSchema.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; public class LongDimensionSchema extends DimensionSchema @@ -43,8 +43,8 @@ public String getTypeName() @Override @JsonIgnore - public ValueType getValueType() + public ColumnType getColumnType() { - return ValueType.LONG; + return ColumnType.LONG; } } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java index 021899590e17..3357a30d5cf4 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/NewSpatialDimensionSchema.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.List; @@ -62,9 +62,9 @@ public String getTypeName() @Override @JsonIgnore - public ValueType getValueType() + public ColumnType getColumnType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/core/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java b/core/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java index 7a83e0f8b06d..10de88d5e26a 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/StringDimensionSchema.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; public class StringDimensionSchema extends DimensionSchema { @@ -57,8 +57,8 @@ public String getTypeName() @Override @JsonIgnore - public ValueType getValueType() + public ColumnType getColumnType() { - return ValueType.STRING; + return ColumnType.STRING; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java b/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java index 7286c4455e06..08e5e91f077c 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java +++ b/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java @@ -113,7 +113,7 @@ default boolean hasArrayOutput(LambdaExpr lambdaExpr) * @see Expr#getOutputType */ @Nullable - ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args); + ExpressionType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args); /** * Base class for "map" functions, which are a class of {@link ApplyFunction} which take a lambda function that is @@ -130,9 +130,9 @@ public boolean hasArrayOutput(LambdaExpr lambdaExpr) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) { - return ExprType.asArrayType(expr.getOutputType(new LambdaInputBindingInspector(inspector, expr, args))); + return ExpressionType.asArrayType(expr.getOutputType(new LambdaInputBindingInspector(inspector, expr, args))); } /** @@ -145,13 +145,13 @@ ExprEval applyMap(LambdaExpr expr, IndexableMapLambdaObjectBinding bindings) Long[] longsOut = null; Double[] doublesOut = null; - ExprType elementType = null; + ExpressionType elementType = null; for (int i = 0; i < length; i++) { ExprEval evaluated = expr.eval(bindings.withIndex(i)); if (elementType == null) { elementType = evaluated.type(); - switch (elementType) { + switch (elementType.getType()) { case STRING: stringsOut = new String[length]; break; @@ -176,7 +176,7 @@ ExprEval applyMap(LambdaExpr expr, IndexableMapLambdaObjectBinding bindings) ); } - switch (elementType) { + switch (elementType.getType()) { case STRING: return ExprEval.ofStringArray(stringsOut); case LONG: @@ -336,7 +336,7 @@ public boolean hasArrayOutput(LambdaExpr lambdaExpr) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) { // output type is accumulator type, which is last argument return args.get(args.size() - 1).getOutputType(inspector); @@ -496,19 +496,16 @@ public ExprEval apply(LambdaExpr lambdaExpr, List argsExpr, Expr.ObjectBin } SettableLambdaBinding lambdaBinding = new SettableLambdaBinding(lambdaExpr, bindings); - switch (arrayEval.type()) { + switch (arrayEval.elementType().getType()) { case STRING: - case STRING_ARRAY: String[] filteredString = this.filter(arrayEval.asStringArray(), lambdaExpr, lambdaBinding).toArray(String[]::new); return ExprEval.ofStringArray(filteredString); case LONG: - case LONG_ARRAY: Long[] filteredLong = this.filter(arrayEval.asLongArray(), lambdaExpr, lambdaBinding).toArray(Long[]::new); return ExprEval.ofLongArray(filteredLong); case DOUBLE: - case DOUBLE_ARRAY: Double[] filteredDouble = this.filter(arrayEval.asDoubleArray(), lambdaExpr, lambdaBinding).toArray(Double[]::new); return ExprEval.ofDoubleArray(filteredDouble); @@ -539,7 +536,7 @@ public void validateArguments(LambdaExpr lambdaExpr, List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) { // output type is input array type return args.get(0).getOutputType(inspector); @@ -594,9 +591,9 @@ public void validateArguments(LambdaExpr lambdaExpr, List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, LambdaExpr expr, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } public abstract ExprEval match(Object[] values, LambdaExpr expr, SettableLambdaBinding bindings); @@ -945,10 +942,10 @@ public LambdaInputBindingInspector(Expr.InputBindingInspector inspector, LambdaE @Nullable @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { if (lambdaIdentifiers.containsKey(name)) { - return ExprType.elementType(args.get(lambdaIdentifiers.getInt(name)).getOutputType(inspector)); + return ExpressionType.elementType(args.get(lambdaIdentifiers.getInt(name)).getOutputType(inspector)); } return inspector.getType(name); } diff --git a/core/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java b/core/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java index 12f1a8b0ee5b..40556625b4f5 100644 --- a/core/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java @@ -22,6 +22,7 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.math.expr.vector.VectorComparisonProcessors; +import org.apache.druid.segment.column.Types; import javax.annotation.Nullable; import java.util.Objects; @@ -62,11 +63,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } @@ -119,11 +120,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } @@ -176,11 +177,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } @@ -232,11 +233,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } @@ -288,11 +289,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } @@ -344,11 +345,11 @@ protected final double evalDouble(double left, double right) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.isNullOr(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } diff --git a/core/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java b/core/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java index 14a613696c21..0b3fb8eeb39b 100644 --- a/core/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java @@ -25,6 +25,7 @@ import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.math.expr.vector.VectorMathProcessors; import org.apache.druid.math.expr.vector.VectorStringProcessors; +import org.apache.druid.segment.column.Types; import javax.annotation.Nullable; @@ -71,11 +72,11 @@ public boolean canVectorize(InputBindingInspector inspector) @Override public ExprVectorProcessor buildVectorized(VectorInputBindingInspector inspector) { - ExprType type = ExprTypeConversion.operator( + ExpressionType type = ExpressionTypeConversion.operator( left.getOutputType(inspector), right.getOutputType(inspector) ); - if (ExprType.STRING.equals(type)) { + if (Types.is(type, ExprType.STRING)) { return VectorStringProcessors.concat(inspector, left, right); } return VectorMathProcessors.plus(inspector, left, right); diff --git a/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java b/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java index a8dd08bd6825..0facebfaca89 100644 --- a/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java @@ -82,9 +82,9 @@ public BindingAnalysis analyzeInputs() @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprTypeConversion.operator(left.getOutputType(inspector), right.getOutputType(inspector)); + return ExpressionTypeConversion.operator(left.getOutputType(inspector), right.getOutputType(inspector)); } @Override @@ -133,8 +133,8 @@ public ExprEval eval(ObjectBinding bindings) return ExprEval.of(null); } - ExprType type = ExprTypeConversion.autoDetect(leftVal, rightVal); - switch (type) { + ExpressionType type = ExpressionTypeConversion.autoDetect(leftVal, rightVal); + switch (type.getType()) { case STRING: return evalString(leftVal.asString(), rightVal.asString()); case LONG: diff --git a/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java b/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java index 5edb2fe7069e..52ea878a97be 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java @@ -37,11 +37,11 @@ */ abstract class ConstantExpr implements Expr { - final ExprType outputType; + final ExpressionType outputType; @Nullable final T value; - protected ConstantExpr(ExprType outputType, @Nullable T value) + protected ConstantExpr(ExpressionType outputType, @Nullable T value) { this.outputType = outputType; this.value = value; @@ -49,7 +49,7 @@ protected ConstantExpr(ExprType outputType, @Nullable T value) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { // null isn't really a type, so don't claim anything return value == null ? null : outputType; @@ -102,7 +102,7 @@ class LongExpr extends ConstantExpr { LongExpr(Long value) { - super(ExprType.LONG, Preconditions.checkNotNull(value, "value")); + super(ExpressionType.LONG, Preconditions.checkNotNull(value, "value")); } @Override @@ -147,7 +147,7 @@ class NullLongExpr extends ConstantExpr { NullLongExpr() { - super(ExprType.LONG, null); + super(ExpressionType.LONG, null); } @Override @@ -185,7 +185,7 @@ class LongArrayExpr extends ConstantExpr { LongArrayExpr(@Nullable Long[] value) { - super(ExprType.LONG_ARRAY, value); + super(ExpressionType.LONG_ARRAY, value); } @Override @@ -239,7 +239,7 @@ class DoubleExpr extends ConstantExpr { DoubleExpr(Double value) { - super(ExprType.DOUBLE, Preconditions.checkNotNull(value, "value")); + super(ExpressionType.DOUBLE, Preconditions.checkNotNull(value, "value")); } @Override @@ -284,7 +284,7 @@ class NullDoubleExpr extends ConstantExpr { NullDoubleExpr() { - super(ExprType.DOUBLE, null); + super(ExpressionType.DOUBLE, null); } @Override @@ -322,7 +322,7 @@ class DoubleArrayExpr extends ConstantExpr { DoubleArrayExpr(@Nullable Double[] value) { - super(ExprType.DOUBLE_ARRAY, value); + super(ExpressionType.DOUBLE_ARRAY, value); } @Override @@ -376,7 +376,7 @@ class StringExpr extends ConstantExpr { StringExpr(@Nullable String value) { - super(ExprType.STRING, NullHandling.emptyToNullIfNeeded(value)); + super(ExpressionType.STRING, NullHandling.emptyToNullIfNeeded(value)); } @Override @@ -428,7 +428,7 @@ class StringArrayExpr extends ConstantExpr { StringArrayExpr(@Nullable String[] value) { - super(ExprType.STRING_ARRAY, value); + super(ExpressionType.STRING_ARRAY, value); } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/Expr.java b/core/src/main/java/org/apache/druid/math/expr/Expr.java index f277179c36ae..3f3ff14741ab 100644 --- a/core/src/main/java/org/apache/druid/math/expr/Expr.java +++ b/core/src/main/java/org/apache/druid/math/expr/Expr.java @@ -135,7 +135,7 @@ default String getBindingIfIdentifier() BindingAnalysis analyzeInputs(); /** - * Given an {@link InputBindingInspector}, compute what the output {@link ExprType} will be for this expression. + * Given an {@link InputBindingInspector}, compute what the output {@link ExpressionType} will be for this expression. * * In the vectorized expression engine, if {@link #canVectorize(InputBindingInspector)} returns true, a return value * of null MUST ONLY indicate that the expression has all null inputs (non-existent columns) or null constants for @@ -148,7 +148,7 @@ default String getBindingIfIdentifier() * transform expressions at ingestion time */ @Nullable - default ExprType getOutputType(InputBindingInspector inspector) + default ExpressionType getOutputType(InputBindingInspector inspector) { return null; } @@ -188,14 +188,14 @@ default byte[] getCacheKey() interface InputBindingInspector { /** - * Get the {@link ExprType} from the backing store for a given identifier (this is likely a column, but could be other + * Get the {@link ExpressionType} from the backing store for a given identifier (this is likely a column, but could be other * things depending on the backing adapter) */ @Nullable - ExprType getType(String name); + ExpressionType getType(String name); /** - * Check if all provided {@link Expr} can infer the output type as {@link ExprType#isNumeric} with a value of true. + * Check if all provided {@link Expr} can infer the output type as {@link ExpressionType#isNumeric} with a value of true. * * There must be at least one expression with a computable numeric output type for this method to return true. */ @@ -203,7 +203,7 @@ default boolean areNumeric(List args) { boolean numeric = true; for (Expr arg : args) { - ExprType argType = arg.getOutputType(this); + ExpressionType argType = arg.getOutputType(this); if (argType == null) { continue; } @@ -213,7 +213,7 @@ default boolean areNumeric(List args) } /** - * Check if all provided {@link Expr} can infer the output type as {@link ExprType#isNumeric} with a value of true. + * Check if all provided {@link Expr} can infer the output type as {@link ExpressionType#isNumeric} with a value of true. * * There must be at least one expression with a computable numeric output type for this method to return true. */ @@ -223,7 +223,7 @@ default boolean areNumeric(Expr... args) } /** - * Check if all provided {@link Expr} can infer the output type as {@link ExprType#isScalar()} (non-array) with a + * Check if all provided {@link Expr} can infer the output type as {@link ExpressionType#isPrimitive()} (non-array) with a * value of true. * * There must be at least one expression with a computable scalar output type for this method to return true. @@ -232,17 +232,17 @@ default boolean areScalar(List args) { boolean scalar = true; for (Expr arg : args) { - ExprType argType = arg.getOutputType(this); + ExpressionType argType = arg.getOutputType(this); if (argType == null) { continue; } - scalar &= argType.isScalar(); + scalar &= argType.isPrimitive(); } return scalar; } /** - * Check if all provided {@link Expr} can infer the output type as {@link ExprType#isScalar()} (non-array) with a + * Check if all provided {@link Expr} can infer the output type as {@link ExpressionType#isPrimitive()} (non-array) with a * value of true. * * There must be at least one expression with a computable scalar output type for this method to return true. @@ -295,7 +295,7 @@ interface ObjectBinding /** * Mechanism to supply batches of input values to a {@link ExprVectorProcessor} for optimized processing. Mirrors - * the vectorized column selector interfaces, and includes {@link ExprType} information about all input bindings + * the vectorized column selector interfaces, and includes {@link ExpressionType} information about all input bindings * which exist */ interface VectorInputBinding extends VectorInputBindingInspector diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java index 8803583c4c8c..4ab5b10e8f83 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java +++ b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java @@ -84,76 +84,82 @@ public static ExprEval deserialize(ByteBuffer buffer, int offset, ExprType type) buffer.get(stringBytes, 0, length); buffer.position(oldPosition); return of(StringUtils.fromUtf8(stringBytes)); - case LONG_ARRAY: - // | expression type (byte) | array length (int) | array bytes | - final int longArrayLength = buffer.getInt(offset); - offset += Integer.BYTES; - if (longArrayLength < 0) { - return ofLongArray(null); - } - final Long[] longs = new Long[longArrayLength]; - for (int i = 0; i < longArrayLength; i++) { - final byte isNull = buffer.get(offset); - offset += Byte.BYTES; - if (isNull == NullHandling.IS_NOT_NULL_BYTE) { - // | is null (byte) | long bytes | - longs[i] = buffer.getLong(offset); - offset += Long.BYTES; - } else { - // | is null (byte) | - longs[i] = null; - } - } - return ofLongArray(longs); - case DOUBLE_ARRAY: - // | expression type (byte) | array length (int) | array bytes | - final int doubleArrayLength = buffer.getInt(offset); - offset += Integer.BYTES; - if (doubleArrayLength < 0) { - return ofDoubleArray(null); - } - final Double[] doubles = new Double[doubleArrayLength]; - for (int i = 0; i < doubleArrayLength; i++) { - final byte isNull = buffer.get(offset); - offset += Byte.BYTES; - if (isNull == NullHandling.IS_NOT_NULL_BYTE) { - // | is null (byte) | double bytes | - doubles[i] = buffer.getDouble(offset); - offset += Double.BYTES; - } else { - // | is null (byte) | - doubles[i] = null; - } - } - return ofDoubleArray(doubles); - case STRING_ARRAY: - // | expression type (byte) | array length (int) | array bytes | - final int stringArrayLength = buffer.getInt(offset); - offset += Integer.BYTES; - if (stringArrayLength < 0) { - return ofStringArray(null); - } - final String[] stringArray = new String[stringArrayLength]; - for (int i = 0; i < stringArrayLength; i++) { - final int stringElementLength = buffer.getInt(offset); - offset += Integer.BYTES; - if (stringElementLength < 0) { - // | string length (int) | - stringArray[i] = null; - } else { - // | string length (int) | string bytes | - final byte[] stringElementBytes = new byte[stringElementLength]; - final int oldPosition2 = buffer.position(); - buffer.position(offset); - buffer.get(stringElementBytes, 0, stringElementLength); - buffer.position(oldPosition2); - stringArray[i] = StringUtils.fromUtf8(stringElementBytes); - offset += stringElementLength; - } + case ARRAY: + final ExprType elementType = ExprType.fromByte(buffer.get(offset++)); + switch (elementType) { + case LONG: + // | expression type (byte) | array element type (byte) | array length (int) | array bytes | + final int longArrayLength = buffer.getInt(offset); + offset += Integer.BYTES; + if (longArrayLength < 0) { + return ofLongArray(null); + } + final Long[] longs = new Long[longArrayLength]; + for (int i = 0; i < longArrayLength; i++) { + final byte isNull = buffer.get(offset); + offset += Byte.BYTES; + if (isNull == NullHandling.IS_NOT_NULL_BYTE) { + // | is null (byte) | long bytes | + longs[i] = buffer.getLong(offset); + offset += Long.BYTES; + } else { + // | is null (byte) | + longs[i] = null; + } + } + return ofLongArray(longs); + case DOUBLE: + // | expression type (byte) | array element type (byte) | array length (int) | array bytes | + final int doubleArrayLength = buffer.getInt(offset); + offset += Integer.BYTES; + if (doubleArrayLength < 0) { + return ofDoubleArray(null); + } + final Double[] doubles = new Double[doubleArrayLength]; + for (int i = 0; i < doubleArrayLength; i++) { + final byte isNull = buffer.get(offset); + offset += Byte.BYTES; + if (isNull == NullHandling.IS_NOT_NULL_BYTE) { + // | is null (byte) | double bytes | + doubles[i] = buffer.getDouble(offset); + offset += Double.BYTES; + } else { + // | is null (byte) | + doubles[i] = null; + } + } + return ofDoubleArray(doubles); + case STRING: + // | expression type (byte) | array element type (byte) | array length (int) | array bytes | + final int stringArrayLength = buffer.getInt(offset); + offset += Integer.BYTES; + if (stringArrayLength < 0) { + return ofStringArray(null); + } + final String[] stringArray = new String[stringArrayLength]; + for (int i = 0; i < stringArrayLength; i++) { + final int stringElementLength = buffer.getInt(offset); + offset += Integer.BYTES; + if (stringElementLength < 0) { + // | string length (int) | + stringArray[i] = null; + } else { + // | string length (int) | string bytes | + final byte[] stringElementBytes = new byte[stringElementLength]; + final int oldPosition2 = buffer.position(); + buffer.position(offset); + buffer.get(stringElementBytes, 0, stringElementLength); + buffer.position(oldPosition2); + stringArray[i] = StringUtils.fromUtf8(stringElementBytes); + offset += stringElementLength; + } + } + return ofStringArray(stringArray); + default: + throw new UOE("Cannot deserialize expression array of type %s", elementType); } - return ofStringArray(stringArray); default: - throw new UOE("how can this be?"); + throw new UOE("Cannot deserialize expression type %s", type); } } @@ -167,8 +173,8 @@ public static ExprEval deserialize(ByteBuffer buffer, int offset, ExprType type) public static void serialize(ByteBuffer buffer, int position, ExprEval eval, int maxSizeBytes) { int offset = position; - buffer.put(offset++, eval.type().getId()); - switch (eval.type()) { + buffer.put(offset++, eval.type().getType().getId()); + switch (eval.type().getType()) { case LONG: if (eval.isNumericNull()) { buffer.put(offset, NullHandling.IS_NULL_BYTE); @@ -201,95 +207,103 @@ public static void serialize(ByteBuffer buffer, int position, ExprEval eval, buffer.putInt(offset, NULL_LENGTH); } break; - case LONG_ARRAY: - Long[] longs = eval.asLongArray(); - if (longs == null) { - // | expression type (byte) | array length (int) | - checkMaxBytes(eval.type(), 1 + Integer.BYTES, maxSizeBytes); - buffer.putInt(offset, NULL_LENGTH); - } else { - // | expression type (byte) | array length (int) | array bytes | - final int sizeBytes = 1 + Integer.BYTES + (Long.BYTES * longs.length); - checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); - buffer.putInt(offset, longs.length); - offset += Integer.BYTES; - for (Long aLong : longs) { - if (aLong != null) { - buffer.put(offset, NullHandling.IS_NOT_NULL_BYTE); - offset++; - buffer.putLong(offset, aLong); - offset += Long.BYTES; + case ARRAY: + // | expression type (byte) | array type (byte) | array length (int) | array bytes | + buffer.put(offset++, eval.type().getElementType().getType().getId()); + switch (eval.type().getElementType().getType()) { + case LONG: + Long[] longs = eval.asLongArray(); + if (longs == null) { + // | expression type (byte) | array type (byte) | array length (int) | + checkMaxBytes(eval.type(), 2 + Integer.BYTES, maxSizeBytes); + buffer.putInt(offset, NULL_LENGTH); } else { - buffer.put(offset++, NullHandling.IS_NULL_BYTE); + // | expression type (byte) | array type (byte) | array length (int) | array bytes | + final int sizeBytes = 2 + Integer.BYTES + (Long.BYTES * longs.length); + checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); + buffer.putInt(offset, longs.length); + offset += Integer.BYTES; + for (Long aLong : longs) { + if (aLong != null) { + buffer.put(offset, NullHandling.IS_NOT_NULL_BYTE); + offset++; + buffer.putLong(offset, aLong); + offset += Long.BYTES; + } else { + buffer.put(offset++, NullHandling.IS_NULL_BYTE); + } + } } - } - } - break; - case DOUBLE_ARRAY: - Double[] doubles = eval.asDoubleArray(); - if (doubles == null) { - // | expression type (byte) | array length (int) | - checkMaxBytes(eval.type(), 1 + Integer.BYTES, maxSizeBytes); - buffer.putInt(offset, NULL_LENGTH); - } else { - // | expression type (byte) | array length (int) | array bytes | - final int sizeBytes = 1 + Integer.BYTES + (Double.BYTES * doubles.length); - checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); - buffer.putInt(offset, doubles.length); - offset += Integer.BYTES; - - for (Double aDouble : doubles) { - if (aDouble != null) { - buffer.put(offset, NullHandling.IS_NOT_NULL_BYTE); - offset++; - buffer.putDouble(offset, aDouble); - offset += Long.BYTES; + break; + case DOUBLE: + Double[] doubles = eval.asDoubleArray(); + if (doubles == null) { + // | expression type (byte) | array type (byte) | array length (int) | + checkMaxBytes(eval.type(), 2 + Integer.BYTES, maxSizeBytes); + buffer.putInt(offset, NULL_LENGTH); } else { - buffer.put(offset++, NullHandling.IS_NULL_BYTE); - } - } - } - break; - case STRING_ARRAY: - String[] strings = eval.asStringArray(); - if (strings == null) { - // | expression type (byte) | array length (int) | - checkMaxBytes(eval.type(), 1 + Integer.BYTES, maxSizeBytes); - buffer.putInt(offset, NULL_LENGTH); - } else { - // | expression type (byte) | array length (int) | array bytes | - buffer.putInt(offset, strings.length); - offset += Integer.BYTES; - int sizeBytes = 1 + Integer.BYTES; - for (String string : strings) { - if (string == null) { - // | string length (int) | - sizeBytes += Integer.BYTES; + // | expression type (byte) | array type (byte) | array length (int) | array bytes | + final int sizeBytes = 2 + Integer.BYTES + (Double.BYTES * doubles.length); checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); - buffer.putInt(offset, NULL_LENGTH); + buffer.putInt(offset, doubles.length); offset += Integer.BYTES; + + for (Double aDouble : doubles) { + if (aDouble != null) { + buffer.put(offset, NullHandling.IS_NOT_NULL_BYTE); + offset++; + buffer.putDouble(offset, aDouble); + offset += Long.BYTES; + } else { + buffer.put(offset++, NullHandling.IS_NULL_BYTE); + } + } + } + break; + case STRING: + String[] strings = eval.asStringArray(); + if (strings == null) { + // | expression type (byte) | array type (byte) | array length (int) | + checkMaxBytes(eval.type(), 2 + Integer.BYTES, maxSizeBytes); + buffer.putInt(offset, NULL_LENGTH); } else { - // | string length (int) | string bytes | - final byte[] stringElementBytes = StringUtils.toUtf8(string); - sizeBytes += Integer.BYTES + stringElementBytes.length; - checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); - buffer.putInt(offset, stringElementBytes.length); + // | expression type (byte) | array type (byte) | array length (int) | array bytes | + buffer.putInt(offset, strings.length); offset += Integer.BYTES; - final int oldPosition = buffer.position(); - buffer.position(offset); - buffer.put(stringElementBytes, 0, stringElementBytes.length); - buffer.position(oldPosition); - offset += stringElementBytes.length; + int sizeBytes = 2 + Integer.BYTES; + for (String string : strings) { + if (string == null) { + // | string length (int) | + sizeBytes += Integer.BYTES; + checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); + buffer.putInt(offset, NULL_LENGTH); + offset += Integer.BYTES; + } else { + // | string length (int) | string bytes | + final byte[] stringElementBytes = StringUtils.toUtf8(string); + sizeBytes += Integer.BYTES + stringElementBytes.length; + checkMaxBytes(eval.type(), sizeBytes, maxSizeBytes); + buffer.putInt(offset, stringElementBytes.length); + offset += Integer.BYTES; + final int oldPosition = buffer.position(); + buffer.position(offset); + buffer.put(stringElementBytes, 0, stringElementBytes.length); + buffer.position(oldPosition); + offset += stringElementBytes.length; + } + } } - } + break; + default: + throw new UOE("Cannot serialize expression array type %s", eval.type()); } break; default: - throw new UOE("how can this be?"); + throw new UOE("Cannot serialize expression type %s", eval.type()); } } - public static void checkMaxBytes(ExprType type, int sizeBytes, int maxSizeBytes) + public static void checkMaxBytes(ExpressionType type, int sizeBytes, int maxSizeBytes) { if (sizeBytes > maxSizeBytes) { throw new ISE("Unable to serialize [%s], size [%s] is larger than max [%s]", type, sizeBytes, maxSizeBytes); @@ -303,7 +317,7 @@ public static void checkMaxBytes(ExprType type, int sizeBytes, int maxSizeBytes) public static void estimateAndCheckMaxBytes(ExprEval eval, int maxSizeBytes) { final int estimated; - switch (eval.type()) { + switch (eval.type().getType()) { case STRING: String stringValue = eval.asString(); estimated = 1 + Integer.BYTES + (stringValue == null ? 0 : StringUtils.estimatedBinaryLengthAsUTF8(stringValue)); @@ -312,45 +326,51 @@ public static void estimateAndCheckMaxBytes(ExprEval eval, int maxSizeBytes) case DOUBLE: estimated = 1 + (NullHandling.sqlCompatible() ? 1 + Long.BYTES : Long.BYTES); break; - case STRING_ARRAY: - String[] stringArray = eval.asStringArray(); - if (stringArray == null) { - estimated = 1 + Integer.BYTES; - } else { - final int elementsSize = Arrays.stream(stringArray) - .filter(Objects::nonNull) - .mapToInt(StringUtils::estimatedBinaryLengthAsUTF8) - .sum(); - // since each value is variably sized, there is an integer per element - estimated = 1 + Integer.BYTES + (Integer.BYTES * stringArray.length) + elementsSize; - } - break; - case LONG_ARRAY: - Long[] longArray = eval.asLongArray(); - if (longArray == null) { - estimated = 1 + Integer.BYTES; - } else { - final int elementsSize = Arrays.stream(longArray) - .filter(Objects::nonNull) - .mapToInt(x -> Long.BYTES) - .sum(); - estimated = 1 + Integer.BYTES + (NullHandling.sqlCompatible() ? longArray.length : 0) + elementsSize; - } - break; - case DOUBLE_ARRAY: - Double[] doubleArray = eval.asDoubleArray(); - if (doubleArray == null) { - estimated = 1 + Integer.BYTES; - } else { - final int elementsSize = Arrays.stream(doubleArray) - .filter(Objects::nonNull) - .mapToInt(x -> Long.BYTES) - .sum(); - estimated = 1 + Integer.BYTES + (NullHandling.sqlCompatible() ? doubleArray.length : 0) + elementsSize; + case ARRAY: + switch (eval.type().getElementType().getType()) { + case STRING: + String[] stringArray = eval.asStringArray(); + if (stringArray == null) { + estimated = 2 + Integer.BYTES; + } else { + final int elementsSize = Arrays.stream(stringArray) + .filter(Objects::nonNull) + .mapToInt(StringUtils::estimatedBinaryLengthAsUTF8) + .sum(); + // since each value is variably sized, there is an integer per element + estimated = 2 + Integer.BYTES + (Integer.BYTES * stringArray.length) + elementsSize; + } + break; + case LONG: + Long[] longArray = eval.asLongArray(); + if (longArray == null) { + estimated = 2 + Integer.BYTES; + } else { + final int elementsSize = Arrays.stream(longArray) + .filter(Objects::nonNull) + .mapToInt(x -> Long.BYTES) + .sum(); + estimated = 2 + Integer.BYTES + (NullHandling.sqlCompatible() ? longArray.length : 0) + elementsSize; + } + break; + case DOUBLE: + Double[] doubleArray = eval.asDoubleArray(); + if (doubleArray == null) { + estimated = 2 + Integer.BYTES; + } else { + final int elementsSize = Arrays.stream(doubleArray) + .filter(Objects::nonNull) + .mapToInt(x -> Long.BYTES) + .sum(); + estimated = 2 + Integer.BYTES + (NullHandling.sqlCompatible() ? doubleArray.length : 0) + elementsSize; + } + break; + default: + throw new ISE("Unsupported array type: %s", eval.type()); } break; default: - throw new IllegalStateException("impossible"); + throw new ISE("Unsupported type: %s", eval.type()); } checkMaxBytes(eval.type(), estimated, maxSizeBytes); } @@ -593,7 +613,12 @@ private ExprEval(@Nullable T value) this.value = value; } - public abstract ExprType type(); + public abstract ExpressionType type(); + + public ExpressionType elementType() + { + return type().isArray() ? (ExpressionType) type().getElementType() : type(); + } @Nullable public T value() @@ -691,7 +716,7 @@ public boolean isArray() @Nullable public abstract Double[] asDoubleArray(); - public abstract ExprEval castTo(ExprType castTo); + public abstract ExprEval castTo(ExpressionType castTo); public abstract Expr toExpr(); @@ -758,9 +783,9 @@ private DoubleExprEval(@Nullable Number value) } @Override - public final ExprType type() + public final ExpressionType type() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override @@ -777,9 +802,9 @@ public Object[] asArray() } @Override - public final ExprEval castTo(ExprType castTo) + public final ExprEval castTo(ExpressionType castTo) { - switch (castTo) { + switch (castTo.getType()) { case DOUBLE: return this; case LONG: @@ -790,12 +815,15 @@ public final ExprEval castTo(ExprType castTo) } case STRING: return ExprEval.of(asString()); - case DOUBLE_ARRAY: - return ExprEval.ofDoubleArray(asDoubleArray()); - case LONG_ARRAY: - return ExprEval.ofLongArray(asLongArray()); - case STRING_ARRAY: - return ExprEval.ofStringArray(asStringArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case DOUBLE: + return ExprEval.ofDoubleArray(asDoubleArray()); + case LONG: + return ExprEval.ofLongArray(asLongArray()); + case STRING: + return ExprEval.ofStringArray(asStringArray()); + } } throw new IAE("invalid type " + castTo); } @@ -820,9 +848,9 @@ private LongExprEval(@Nullable Number value) } @Override - public final ExprType type() + public final ExpressionType type() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -846,9 +874,9 @@ public Long[] asLongArray() } @Override - public final ExprEval castTo(ExprType castTo) + public final ExprEval castTo(ExpressionType castTo) { - switch (castTo) { + switch (castTo.getType()) { case DOUBLE: if (value == null) { return ExprEval.ofDouble(null); @@ -859,12 +887,15 @@ public final ExprEval castTo(ExprType castTo) return this; case STRING: return ExprEval.of(asString()); - case DOUBLE_ARRAY: - return ExprEval.ofDoubleArray(asDoubleArray()); - case LONG_ARRAY: - return ExprEval.ofLongArray(asLongArray()); - case STRING_ARRAY: - return ExprEval.ofStringArray(asStringArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case DOUBLE: + return ExprEval.ofDoubleArray(asDoubleArray()); + case LONG: + return ExprEval.ofLongArray(asLongArray()); + case STRING: + return ExprEval.ofStringArray(asStringArray()); + } } throw new IAE("invalid type " + castTo); } @@ -902,9 +933,9 @@ private StringExprEval(@Nullable String value) } @Override - public final ExprType type() + public final ExpressionType type() { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -1037,21 +1068,24 @@ public Double[] asDoubleArray() } @Override - public final ExprEval castTo(ExprType castTo) + public final ExprEval castTo(ExpressionType castTo) { - switch (castTo) { + switch (castTo.getType()) { case DOUBLE: return ExprEval.ofDouble(computeNumber()); case LONG: return ExprEval.ofLong(computeNumber()); case STRING: return this; - case DOUBLE_ARRAY: - return ExprEval.ofDoubleArray(asDoubleArray()); - case LONG_ARRAY: - return ExprEval.ofLongArray(asLongArray()); - case STRING_ARRAY: - return ExprEval.ofStringArray(asStringArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case DOUBLE: + return ExprEval.ofDoubleArray(asDoubleArray()); + case LONG: + return ExprEval.ofLongArray(asLongArray()); + case STRING: + return ExprEval.ofStringArray(asStringArray()); + } } throw new IAE("invalid type " + castTo); } @@ -1167,9 +1201,9 @@ private LongArrayExprEval(@Nullable Long[] value) } @Override - public ExprType type() + public ExpressionType type() { - return ExprType.LONG_ARRAY; + return ExpressionType.LONG_ARRAY; } @Override @@ -1250,12 +1284,12 @@ public Double[] asDoubleArray() } @Override - public ExprEval castTo(ExprType castTo) + public ExprEval castTo(ExpressionType castTo) { if (value == null) { return StringExprEval.OF_NULL; } - switch (castTo) { + switch (castTo.getType()) { case STRING: if (value.length == 1) { return ExprEval.of(asString()); @@ -1271,12 +1305,16 @@ public ExprEval castTo(ExprType castTo) return isNumericNull() ? ExprEval.ofDouble(null) : ExprEval.ofDouble(asDouble()); } break; - case LONG_ARRAY: - return this; - case DOUBLE_ARRAY: - return ExprEval.ofDoubleArray(asDoubleArray()); - case STRING_ARRAY: - return ExprEval.ofStringArray(asStringArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case LONG: + return this; + case DOUBLE: + return ExprEval.ofDoubleArray(asDoubleArray()); + case STRING: + return ExprEval.ofStringArray(asStringArray()); + } + break; } throw new IAE("invalid type " + castTo); @@ -1299,9 +1337,9 @@ private DoubleArrayExprEval(@Nullable Double[] value) } @Override - public ExprType type() + public ExpressionType type() { - return ExprType.DOUBLE_ARRAY; + return ExpressionType.DOUBLE_ARRAY; } @Override @@ -1384,12 +1422,12 @@ public Double[] asDoubleArray() } @Override - public ExprEval castTo(ExprType castTo) + public ExprEval castTo(ExpressionType castTo) { if (value == null) { return StringExprEval.OF_NULL; } - switch (castTo) { + switch (castTo.getType()) { case STRING: if (value.length == 1) { return ExprEval.of(asString()); @@ -1405,12 +1443,15 @@ public ExprEval castTo(ExprType castTo) return isNumericNull() ? ExprEval.ofDouble(null) : ExprEval.ofDouble(asDouble()); } break; - case LONG_ARRAY: - return ExprEval.ofLongArray(asLongArray()); - case DOUBLE_ARRAY: - return this; - case STRING_ARRAY: - return ExprEval.ofStringArray(asStringArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case LONG: + return ExprEval.ofLongArray(asLongArray()); + case DOUBLE: + return this; + case STRING: + return ExprEval.ofStringArray(asStringArray()); + } } throw new IAE("invalid type " + castTo); @@ -1443,9 +1484,9 @@ private StringArrayExprEval(@Nullable String[] value) } @Override - public ExprType type() + public ExpressionType type() { - return ExprType.STRING_ARRAY; + return ExpressionType.STRING_ARRAY; } @Override @@ -1539,12 +1580,12 @@ public Double[] asDoubleArray() } @Override - public ExprEval castTo(ExprType castTo) + public ExprEval castTo(ExpressionType castTo) { if (value == null) { return StringExprEval.OF_NULL; } - switch (castTo) { + switch (castTo.getType()) { case STRING: if (value.length == 1) { return ExprEval.of(asString()); @@ -1560,12 +1601,15 @@ public ExprEval castTo(ExprType castTo) return isNumericNull() ? ExprEval.ofDouble(null) : ExprEval.ofDouble(asDouble()); } break; - case STRING_ARRAY: - return this; - case LONG_ARRAY: - return ExprEval.ofLongArray(asLongArray()); - case DOUBLE_ARRAY: - return ExprEval.ofDoubleArray(asDoubleArray()); + case ARRAY: + switch (castTo.getElementType().getType()) { + case STRING: + return this; + case LONG: + return ExprEval.ofLongArray(asLongArray()); + case DOUBLE: + return ExprEval.ofDoubleArray(asDoubleArray()); + } } throw new IAE("invalid type " + castTo); } diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprType.java b/core/src/main/java/org/apache/druid/math/expr/ExprType.java index 80b5e0365b43..f0e25764e94b 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ExprType.java +++ b/core/src/main/java/org/apache/druid/math/expr/ExprType.java @@ -21,22 +21,18 @@ import it.unimi.dsi.fastutil.bytes.Byte2ObjectArrayMap; import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.segment.column.ValueType; - -import javax.annotation.Nullable; +import org.apache.druid.segment.column.TypeDescriptor; /** * Base 'value' types of Druid expression language, all {@link Expr} must evaluate to one of these types. */ -public enum ExprType +public enum ExprType implements TypeDescriptor { DOUBLE((byte) 0x01), LONG((byte) 0x02), STRING((byte) 0x03), - DOUBLE_ARRAY((byte) 0x04), - LONG_ARRAY((byte) 0x05), - STRING_ARRAY((byte) 0x06); + ARRAY((byte) 0x04), + COMPLEX((byte) 0x05); private static final Byte2ObjectMap TYPE_BYTES = new Byte2ObjectArrayMap<>(ExprType.values().length); @@ -58,149 +54,26 @@ public byte getId() return id; } + @Override public boolean isNumeric() { - return isNumeric(this); - } - - public boolean isScalar() - { - return isScalar(this); - } - - public static ExprType fromByte(byte id) - { - return TYPE_BYTES.get(id); - } - - /** - * The expression system does not distinguish between {@link ValueType#FLOAT} and {@link ValueType#DOUBLE}, and - * cannot currently handle {@link ValueType#COMPLEX} inputs. This method will convert {@link ValueType#FLOAT} to - * {@link #DOUBLE}, or throw an exception if a {@link ValueType#COMPLEX} is encountered. - * - * @throws IllegalStateException - */ - public static ExprType fromValueTypeStrict(@Nullable ValueType valueType) - { - if (valueType == null) { - throw new IllegalStateException("Unsupported unknown value type"); - } - switch (valueType) { - case LONG: - return LONG; - case LONG_ARRAY: - return LONG_ARRAY; - case FLOAT: - case DOUBLE: - return DOUBLE; - case DOUBLE_ARRAY: - return DOUBLE_ARRAY; - case STRING: - return STRING; - case STRING_ARRAY: - return STRING_ARRAY; - case COMPLEX: - default: - throw new ISE("Unsupported value type[%s]", valueType); - } - } - - /** - * The expression system does not distinguish between {@link ValueType#FLOAT} and {@link ValueType#DOUBLE}, and - * cannot currently handle {@link ValueType#COMPLEX} inputs. This method will convert {@link ValueType#FLOAT} to - * {@link #DOUBLE}, or null if a null {@link ValueType#COMPLEX} is encountered. - */ - @Nullable - public static ExprType fromValueType(@Nullable ValueType valueType) - { - if (valueType == null) { - return null; - } - switch (valueType) { - case LONG: - return LONG; - case LONG_ARRAY: - return LONG_ARRAY; - case FLOAT: - case DOUBLE: - return DOUBLE; - case DOUBLE_ARRAY: - return DOUBLE_ARRAY; - case STRING: - return STRING; - case STRING_ARRAY: - return STRING_ARRAY; - case COMPLEX: - default: - return null; - } + return LONG.equals(this) || DOUBLE.equals(this); } - - public static ValueType toValueType(ExprType exprType) + @Override + public boolean isPrimitive() { - switch (exprType) { - case LONG: - return ValueType.LONG; - case LONG_ARRAY: - return ValueType.LONG_ARRAY; - case DOUBLE: - return ValueType.DOUBLE; - case DOUBLE_ARRAY: - return ValueType.DOUBLE_ARRAY; - case STRING: - return ValueType.STRING; - case STRING_ARRAY: - return ValueType.STRING_ARRAY; - default: - throw new ISE("Unsupported expression type[%s]", exprType); - } + return this != ARRAY && this != COMPLEX; } - public static boolean isNumeric(@Nullable ExprType type) + @Override + public boolean isArray() { - return LONG.equals(type) || DOUBLE.equals(type); + return this == ExprType.ARRAY; } - public static boolean isScalar(@Nullable ExprType exprType) - { - return !isArray(exprType); - } - - public static boolean isArray(@Nullable ExprType type) - { - return LONG_ARRAY.equals(type) || DOUBLE_ARRAY.equals(type) || STRING_ARRAY.equals(type); - } - - @Nullable - public static ExprType elementType(@Nullable ExprType type) - { - if (type != null) { - switch (type) { - case STRING_ARRAY: - return STRING; - case LONG_ARRAY: - return LONG; - case DOUBLE_ARRAY: - return DOUBLE; - } - } - return type; - } - - @Nullable - public static ExprType asArrayType(@Nullable ExprType elementType) + public static ExprType fromByte(byte id) { - if (elementType != null) { - switch (elementType) { - case STRING: - return STRING_ARRAY; - case LONG: - return LONG_ARRAY; - case DOUBLE: - return DOUBLE_ARRAY; - } - } - return elementType; + return TYPE_BYTES.get(id); } } diff --git a/core/src/main/java/org/apache/druid/math/expr/ExpressionType.java b/core/src/main/java/org/apache/druid/math/expr/ExpressionType.java new file mode 100644 index 000000000000..635aca05536c --- /dev/null +++ b/core/src/main/java/org/apache/druid/math/expr/ExpressionType.java @@ -0,0 +1,208 @@ +/* + * 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.math.expr; + +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.ISE; +import org.apache.druid.segment.column.BaseTypeSignature; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.Types; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; + +/** + * The type system used to process Druid expressions. This is basically {@link ColumnType}, but without + * {@link ColumnType#FLOAT} because the expression processing system does not currently directly support them. + */ +@JsonSerialize(using = ToStringSerializer.class) +public class ExpressionType extends BaseTypeSignature +{ + public static final ExpressionType STRING = + new ExpressionType(ExprType.STRING, null, null); + public static final ExpressionType LONG = + new ExpressionType(ExprType.LONG, null, null); + public static final ExpressionType DOUBLE = + new ExpressionType(ExprType.DOUBLE, null, null); + public static final ExpressionType STRING_ARRAY = + new ExpressionType(ExprType.ARRAY, null, STRING); + public static final ExpressionType LONG_ARRAY = + new ExpressionType(ExprType.ARRAY, null, LONG); + public static final ExpressionType DOUBLE_ARRAY = + new ExpressionType(ExprType.ARRAY, null, DOUBLE); + + @JsonCreator + public ExpressionType( + @JsonProperty("type") ExprType exprType, + @JsonProperty("complexTypeName") @Nullable String complexTypeName, + @JsonProperty("elementType") @Nullable ExpressionType elementType + ) + { + super(exprType, complexTypeName, elementType); + } + + @Nullable + @JsonCreator + public static ExpressionType fromString(@Nullable String typeName) + { + return Types.fromString(ExpressionTypeFactory.getInstance(), typeName); + } + + /** + * If an {@link ExpressionType} is an array, return {@link ExpressionType#getElementType()}, otherwise the type is + * returned unchanged. + */ + @Nullable + public static ExpressionType elementType(@Nullable ExpressionType type) + { + if (type != null && type.isArray()) { + return (ExpressionType) type.getElementType(); + } + return type; + } + + /** + * Convert a primitive {@link ExpressionType} into an array of that type. Non-primitive types are passed through, + * even if they are not arrays. + */ + @Nullable + public static ExpressionType asArrayType(@Nullable ExpressionType elementType) + { + if (elementType != null && elementType.isPrimitive()) { + switch (elementType.getType()) { + case STRING: + return STRING_ARRAY; + case LONG: + return LONG_ARRAY; + case DOUBLE: + return DOUBLE_ARRAY; + } + } + return elementType; + } + + + /** + * The expression system does not distinguish between {@link ValueType#FLOAT} and {@link ValueType#DOUBLE}, so, + * this method will convert {@link ValueType#FLOAT} to {@link #DOUBLE}. Null values are not allowed in this method, + * and will result in an {@link IllegalStateException} + * + * @throws IllegalStateException + */ + public static ExpressionType fromColumnTypeStrict(@Nullable TypeSignature valueType) + { + if (valueType == null) { + throw new IllegalStateException("Unsupported unknown value type"); + } + switch (valueType.getType()) { + case LONG: + return LONG; + case FLOAT: + case DOUBLE: + return DOUBLE; + case STRING: + return STRING; + case ARRAY: + switch (valueType.getElementType().getType()) { + case LONG: + return LONG_ARRAY; + case DOUBLE: + return DOUBLE_ARRAY; + case STRING: + return STRING_ARRAY; + } + return ExpressionTypeFactory.getInstance().ofArray(fromColumnTypeStrict(valueType.getElementType())); + case COMPLEX: + return ExpressionTypeFactory.getInstance().ofComplex(valueType.getComplexTypeName()); + default: + throw new ISE("Unsupported value type[%s]", valueType); + } + } + + + /** + * The expression system does not distinguish between {@link ValueType#FLOAT} and {@link ValueType#DOUBLE}, so this + * method will convert {@link ValueType#FLOAT} to {@link #DOUBLE}. + */ + @Nullable + public static ExpressionType fromColumnType(@Nullable TypeSignature valueType) + { + if (valueType == null) { + return null; + } + switch (valueType.getType()) { + case LONG: + return LONG; + case FLOAT: + case DOUBLE: + return DOUBLE; + case STRING: + return STRING; + case ARRAY: + switch (valueType.getElementType().getType()) { + case LONG: + return LONG_ARRAY; + case DOUBLE: + return DOUBLE_ARRAY; + case STRING: + return STRING_ARRAY; + } + return ExpressionTypeFactory.getInstance().ofArray(fromColumnType(valueType.getElementType())); + case COMPLEX: + return ExpressionTypeFactory.getInstance().ofComplex(valueType.getComplexTypeName()); + default: + return null; + } + } + + /** + * Convert {@link ExpressionType} to the corresponding {@link ColumnType} + */ + public static ColumnType toColumnType(ExpressionType exprType) + { + switch (exprType.getType()) { + case LONG: + return ColumnType.LONG; + case DOUBLE: + return ColumnType.DOUBLE; + case STRING: + return ColumnType.STRING; + case ARRAY: + switch (exprType.getElementType().getType()) { + case LONG: + return ColumnType.LONG_ARRAY; + case DOUBLE: + return ColumnType.DOUBLE_ARRAY; + case STRING: + return ColumnType.STRING_ARRAY; + default: + return ColumnType.ofArray(toColumnType((ExpressionType) exprType.getElementType())); + } + case COMPLEX: + return ColumnType.ofComplex(exprType.getComplexTypeName()); + default: + throw new ISE("Unsupported expression type[%s]", exprType); + } + } +} diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprTypeConversion.java b/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java similarity index 70% rename from core/src/main/java/org/apache/druid/math/expr/ExprTypeConversion.java rename to core/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java index a5bd093ad777..98a4b094b098 100644 --- a/core/src/main/java/org/apache/druid/math/expr/ExprTypeConversion.java +++ b/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java @@ -20,19 +20,21 @@ package org.apache.druid.math.expr; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.column.Types; import javax.annotation.Nullable; import java.util.List; +import java.util.Objects; -public class ExprTypeConversion +public class ExpressionTypeConversion { /** * Infer the output type of a list of possible 'conditional' expression outputs (where any of these could be the * output expression if the corresponding case matching expression evaluates to true) */ - public static ExprType conditional(Expr.InputBindingInspector inspector, List args) + public static ExpressionType conditional(Expr.InputBindingInspector inspector, List args) { - ExprType type = null; + ExpressionType type = null; for (Expr arg : args) { if (type == null) { type = arg.getOutputType(inspector); @@ -49,12 +51,12 @@ public static ExprType conditional(Expr.InputBindingInspector inspector, List +{ + private static final ExpressionTypeFactory INSTANCE = new ExpressionTypeFactory(); + private static final Interner INTERNER = Interners.newWeakInterner(); + + public static ExpressionTypeFactory getInstance() + { + return INSTANCE; + } + + private ExpressionTypeFactory() + { + // no instantiation + } + + @Override + public ExpressionType ofString() + { + return ExpressionType.STRING; + } + + @Override + public ExpressionType ofFloat() + { + throw new IllegalStateException("FLOAT types are not supported natively by Druid expressions"); + } + + @Override + public ExpressionType ofDouble() + { + return ExpressionType.DOUBLE; + } + + @Override + public ExpressionType ofLong() + { + return ExpressionType.LONG; + } + + @Override + public ExpressionType ofArray(ExpressionType elementType) + { + if (elementType.isPrimitive()) { + switch (elementType.getType()) { + case STRING: + return ExpressionType.STRING_ARRAY; + case DOUBLE: + return ExpressionType.DOUBLE_ARRAY; + case LONG: + return ExpressionType.LONG_ARRAY; + } + } + return INTERNER.intern(new ExpressionType(ExprType.ARRAY, null, elementType)); + } + + @Override + public ExpressionType ofComplex(@Nullable String complexTypeName) + { + return INTERNER.intern(new ExpressionType(ExprType.COMPLEX, complexTypeName, null)); + } +} diff --git a/core/src/main/java/org/apache/druid/math/expr/Function.java b/core/src/main/java/org/apache/druid/math/expr/Function.java index 99f9eac1dfbb..e73749e138b9 100644 --- a/core/src/main/java/org/apache/druid/math/expr/Function.java +++ b/core/src/main/java/org/apache/druid/math/expr/Function.java @@ -118,7 +118,7 @@ default boolean hasArrayOutput() * @see Expr#getOutputType */ @Nullable - ExprType getOutputType(Expr.InputBindingInspector inspector, List args); + ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args); /** * Check if a function can be 'vectorized', for a given set of {@link Expr} inputs. If this method returns true, @@ -204,9 +204,9 @@ protected final ExprEval eval(ExprEval param) if (NullHandling.sqlCompatible() && param.isNumericNull()) { return ExprEval.of(null); } - if (param.type() == ExprType.LONG) { + if (param.type().is(ExprType.LONG)) { return eval(param.asLong()); - } else if (param.type() == ExprType.DOUBLE) { + } else if (param.type().is(ExprType.DOUBLE)) { return eval(param.asDouble()); } return ExprEval.of(null); @@ -227,7 +227,7 @@ protected ExprEval eval(double param) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { return args.get(0).getOutputType(inspector); } @@ -246,9 +246,9 @@ abstract class DoubleUnivariateMathFunction extends UnivariateMathFunction { @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } } @@ -266,8 +266,8 @@ protected final ExprEval eval(ExprEval x, ExprEval y) return ExprEval.of(null); } - ExprType type = ExprTypeConversion.autoDetect(x, y); - switch (type) { + ExpressionType type = ExpressionTypeConversion.autoDetect(x, y); + switch (type.getType()) { case STRING: return ExprEval.of(null); case LONG: @@ -290,9 +290,9 @@ protected ExprEval eval(double x, double y) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprTypeConversion.function( + return ExpressionTypeConversion.function( args.get(0).getOutputType(inspector), args.get(1).getOutputType(inspector) ); @@ -312,9 +312,9 @@ abstract class DoubleBivariateMathFunction extends BivariateMathFunction { @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } } @@ -329,8 +329,8 @@ protected final ExprEval eval(ExprEval x, ExprEval y) return ExprEval.of(null); } - ExprType type = ExprTypeConversion.autoDetect(x, y); - if (type == ExprType.STRING) { + ExpressionType type = ExpressionTypeConversion.autoDetect(x, y); + if (type.is(ExprType.STRING)) { return ExprEval.of(null); } return eval(x.asLong(), y.asLong()); @@ -340,9 +340,9 @@ protected final ExprEval eval(ExprEval x, ExprEval y) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -361,7 +361,7 @@ abstract class StringLongFunction extends BivariateFunction @Override protected final ExprEval eval(ExprEval x, ExprEval y) { - if (x.type() != ExprType.STRING || y.type() != ExprType.LONG) { + if (!x.type().is(ExprType.STRING) || !y.type().is(ExprType.LONG)) { throw new IAE( "Function[%s] needs a string as first argument and an integer as second argument", name() @@ -492,21 +492,19 @@ public boolean hasArrayOutput() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType arrayType = getArrayArgument(args).getOutputType(inspector); - return Optional.ofNullable(ExprType.asArrayType(arrayType)).orElse(arrayType); + ExpressionType arrayType = getArrayArgument(args).getOutputType(inspector); + return Optional.ofNullable(ExpressionType.asArrayType(arrayType)).orElse(arrayType); } @Override ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr) { - switch (arrayExpr.type()) { + switch (arrayExpr.elementType().getType()) { case STRING: - case STRING_ARRAY: return ExprEval.ofStringArray(add(arrayExpr.asStringArray(), scalarExpr.asString()).toArray(String[]::new)); case LONG: - case LONG_ARRAY: return ExprEval.ofLongArray( add( arrayExpr.asLongArray(), @@ -514,7 +512,6 @@ ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr) ).toArray(Long[]::new) ); case DOUBLE: - case DOUBLE_ARRAY: return ExprEval.ofDoubleArray( add( arrayExpr.asDoubleArray(), @@ -548,10 +545,10 @@ public boolean hasArrayOutput() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType arrayType = args.get(0).getOutputType(inspector); - return Optional.ofNullable(ExprType.asArrayType(arrayType)).orElse(arrayType); + ExpressionType arrayType = args.get(0).getOutputType(inspector); + return Optional.ofNullable(ExpressionType.asArrayType(arrayType)).orElse(arrayType); } @Override @@ -567,19 +564,16 @@ ExprEval doApply(ExprEval lhsExpr, ExprEval rhsExpr) return lhsExpr; } - switch (lhsExpr.type()) { + switch (lhsExpr.elementType().getType()) { case STRING: - case STRING_ARRAY: return ExprEval.ofStringArray( merge(lhsExpr.asStringArray(), rhsExpr.asStringArray()).toArray(String[]::new) ); case LONG: - case LONG_ARRAY: return ExprEval.ofLongArray( merge(lhsExpr.asLongArray(), rhsExpr.asLongArray()).toArray(Long[]::new) ); case DOUBLE: - case DOUBLE_ARRAY: return ExprEval.ofDoubleArray( merge(lhsExpr.asDoubleArray(), rhsExpr.asDoubleArray()).toArray(Double[]::new) ); @@ -615,11 +609,11 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType outputType = ExprType.LONG; + ExpressionType outputType = ExpressionType.LONG; for (Expr expr : args) { - outputType = ExprTypeConversion.function(outputType, expr.getOutputType(inspector)); + outputType = ExpressionTypeConversion.function(outputType, expr.getOutputType(inspector)); } return outputType; } @@ -633,14 +627,14 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) // evaluate arguments and collect output type List> evals = new ArrayList<>(); - ExprType outputType = ExprType.LONG; + ExpressionType outputType = ExpressionType.LONG; for (Expr expr : args) { ExprEval exprEval = expr.eval(bindings); - ExprType exprType = exprEval.type(); + ExpressionType exprType = exprEval.type(); if (isValidType(exprType)) { - outputType = ExprTypeConversion.function(outputType, exprType); + outputType = ExpressionTypeConversion.function(outputType, exprType); } if (exprEval.value() != null) { @@ -658,7 +652,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return ExprEval.of(null); } - switch (outputType) { + switch (outputType.getType()) { case DOUBLE: //noinspection OptionalGetWithoutIsPresent (empty list handled earlier) return ExprEval.of(evals.stream().mapToDouble(ExprEval::asDouble).reduce(doubleReducer).getAsDouble()); @@ -671,9 +665,9 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) } } - private boolean isValidType(ExprType exprType) + private boolean isValidType(ExpressionType exprType) { - switch (exprType) { + switch (exprType.getType()) { case DOUBLE: case LONG: case STRING: @@ -704,9 +698,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -781,9 +775,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override @@ -899,9 +893,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -927,13 +921,13 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType type = args.get(0).getOutputType(inspector); + ExpressionType type = args.get(0).getOutputType(inspector); if (type == null) { return null; } - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override @@ -959,13 +953,13 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType type = args.get(0).getOutputType(inspector); + ExpressionType type = args.get(0).getOutputType(inspector); if (type == null) { return null; } - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -1213,9 +1207,9 @@ protected ExprEval eval(final double x, final double y) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprTypeConversion.integerMathFunction( + return ExpressionTypeConversion.integerMathFunction( args.get(0).getOutputType(inspector), args.get(1).getOutputType(inspector) ); @@ -1307,9 +1301,9 @@ protected ExprEval eval(double param) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -1446,7 +1440,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return ExprEval.of(null); } - if (value1.type() != ExprType.LONG && value1.type() != ExprType.DOUBLE) { + if (!value1.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) { throw new IAE( "The first argument to the function[%s] should be integer or double type but got the type: %s", name(), @@ -1458,7 +1452,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return eval(value1); } else { ExprEval value2 = args.get(1).eval(bindings); - if (value2.type() != ExprType.LONG) { + if (!value2.type().is(ExprType.LONG)) { throw new IAE( "The second argument to the function[%s] should be integer type but got the type: %s", name(), @@ -1479,7 +1473,7 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { return args.get(0).getOutputType(inspector); } @@ -1491,9 +1485,9 @@ private ExprEval eval(ExprEval param) private ExprEval eval(ExprEval param, int scale) { - if (param.type() == ExprType.LONG) { + if (param.type().is(ExprType.LONG)) { return ExprEval.of(BigDecimal.valueOf(param.asLong()).setScale(scale, RoundingMode.HALF_UP).longValue()); - } else if (param.type() == ExprType.DOUBLE) { + } else if (param.type().is(ExprType.DOUBLE)) { BigDecimal decimal = safeGetFromDouble(param.asDouble()); return ExprEval.of(decimal.setScale(scale, RoundingMode.HALF_UP).doubleValue()); } else { @@ -1896,9 +1890,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override @@ -1908,8 +1902,8 @@ protected ExprEval eval(ExprEval x, ExprEval y) return ExprEval.of(null); } - ExprType type = ExprTypeConversion.autoDetect(x, y); - switch (type) { + ExpressionType type = ExpressionTypeConversion.autoDetect(x, y); + switch (type.getType()) { case STRING: return ExprEval.of(null); default: @@ -1944,9 +1938,9 @@ protected ExprEval eval(ExprEval x, ExprEval y) if (NullHandling.sqlCompatible() && x.value() == null) { return ExprEval.of(null); } - ExprType castTo; + ExpressionType castTo; try { - castTo = ExprType.valueOf(StringUtils.toUpperCase(y.asString())); + castTo = ExpressionType.fromString(StringUtils.toUpperCase(y.asString())); } catch (IllegalArgumentException e) { throw new IAE("invalid type '%s'", y.asString()); @@ -1958,11 +1952,9 @@ protected ExprEval eval(ExprEval x, ExprEval y) public Set getScalarInputs(List args) { if (args.get(1).isLiteral()) { - ExprType castTo = ExprType.valueOf(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); - switch (castTo) { - case LONG_ARRAY: - case DOUBLE_ARRAY: - case STRING_ARRAY: + ExpressionType castTo = ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); + switch (castTo.getType()) { + case ARRAY: return Collections.emptySet(); default: return ImmutableSet.of(args.get(0)); @@ -1976,8 +1968,8 @@ public Set getScalarInputs(List args) public Set getArrayInputs(List args) { if (args.get(1).isLiteral()) { - ExprType castTo = ExprType.valueOf(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); - switch (castTo) { + ExpressionType castTo = ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); + switch (castTo.getType()) { case LONG: case DOUBLE: case STRING: @@ -1992,11 +1984,11 @@ public Set getArrayInputs(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { // can only know cast output type if cast to argument is constant if (args.get(1).isLiteral()) { - return ExprType.valueOf(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); + return ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())); } return null; } @@ -2012,7 +2004,7 @@ public ExprVectorProcessor asVectorProcessor(Expr.VectorInputBindingInspe { return CastToTypeVectorProcessor.cast( args.get(0).buildVectorized(inspector), - ExprType.valueOf(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())) + ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())) ); } } @@ -2082,9 +2074,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprTypeConversion.conditional(inspector, args.subList(1, 3)); + return ExpressionTypeConversion.conditional(inspector, args.subList(1, 3)); } } @@ -2125,7 +2117,7 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { List results = new ArrayList<>(); for (int i = 1; i < args.size(); i += 2) { @@ -2133,7 +2125,7 @@ public ExprType getOutputType(Expr.InputBindingInspector inspector, List a } // add else results.add(args.get(args.size() - 1)); - return ExprTypeConversion.conditional(inspector, results); + return ExpressionTypeConversion.conditional(inspector, results); } } @@ -2174,7 +2166,7 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { List results = new ArrayList<>(); for (int i = 2; i < args.size(); i += 2) { @@ -2182,7 +2174,7 @@ public ExprType getOutputType(Expr.InputBindingInspector inspector, List a } // add else results.add(args.get(args.size() - 1)); - return ExprTypeConversion.conditional(inspector, results); + return ExpressionTypeConversion.conditional(inspector, results); } } @@ -2211,9 +2203,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprTypeConversion.conditional(inspector, args); + return ExpressionTypeConversion.conditional(inspector, args); } } @@ -2242,9 +2234,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -2273,9 +2265,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -2323,9 +2315,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -2369,9 +2361,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -2410,9 +2402,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2455,9 +2447,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -2505,9 +2497,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2521,9 +2513,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -2553,9 +2545,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -2604,9 +2596,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2638,9 +2630,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2672,9 +2664,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2688,15 +2680,15 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override protected ExprEval eval(ExprEval param) { - if (param.type() != ExprType.STRING) { + if (!param.type().is(ExprType.STRING)) { throw new IAE( "Function[%s] needs a string argument", name() @@ -2717,9 +2709,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -2765,9 +2757,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2804,9 +2796,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } @@ -2822,14 +2814,14 @@ public String name() public ExprEval apply(List args, Expr.ObjectBinding bindings) { ExprEval value = args.get(0).eval(bindings); - if (value.type() != ExprType.STRING) { + if (!value.type().is(ExprType.STRING)) { throw new IAE("first argument should be string type but got %s type", value.type()); } DateTimes.UtcFormatter formatter = DateTimes.ISO_DATE_OPTIONAL_TIME; if (args.size() > 1) { ExprEval format = args.get(1).eval(bindings); - if (format.type() != ExprType.STRING) { + if (!format.type().is(ExprType.STRING)) { throw new IAE("second argument should be string type but got %s type", format.type()); } formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format.asString())); @@ -2854,9 +2846,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } protected ExprEval toValue(DateTime date) @@ -2913,9 +2905,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -2937,12 +2929,12 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) Long[] longsOut = null; Double[] doublesOut = null; - ExprType elementType = null; + ExpressionType elementType = null; for (int i = 0; i < length; i++) { ExprEval evaluated = args.get(i).eval(bindings); if (elementType == null) { elementType = evaluated.type(); - switch (elementType) { + switch (elementType.getType()) { case STRING: stringsOut = new String[length]; break; @@ -2963,7 +2955,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) // There should be always at least one argument and thus elementType is never null. // See validateArguments(). //noinspection ConstantConditions - switch (elementType) { + switch (elementType.getType()) { case STRING: return ExprEval.ofStringArray(stringsOut); case LONG: @@ -2979,12 +2971,12 @@ static void setArrayOutputElement( String[] stringsOut, Long[] longsOut, Double[] doublesOut, - ExprType elementType, + ExpressionType elementType, int i, ExprEval evaluated ) { - switch (elementType) { + switch (elementType.getType()) { case STRING: stringsOut[i] = evaluated.asString(); break; @@ -3026,13 +3018,13 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - ExprType type = ExprType.LONG; + ExpressionType type = ExpressionType.LONG; for (Expr arg : args) { - type = ExprTypeConversion.function(type, arg.getOutputType(inspector)); + type = ExpressionTypeConversion.function(type, arg.getOutputType(inspector)); } - return ExprType.asArrayType(type); + return ExpressionType.asArrayType(type); } } @@ -3082,9 +3074,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -3112,9 +3104,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING_ARRAY; + return ExpressionType.STRING_ARRAY; } @Override @@ -3153,9 +3145,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -3182,9 +3174,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.elementType(args.get(0).getOutputType(inspector)); + return ExpressionType.elementType(args.get(0).getOutputType(inspector)); } @Override @@ -3210,9 +3202,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.elementType(args.get(0).getOutputType(inspector)); + return ExpressionType.elementType(args.get(0).getOutputType(inspector)); } @Override @@ -3238,9 +3230,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -3248,7 +3240,7 @@ ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr) { final Object[] array = arrayExpr.asArray(); - switch (scalarExpr.type()) { + switch (scalarExpr.type().getType()) { case STRING: case LONG: case DOUBLE: @@ -3276,16 +3268,16 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override ExprEval doApply(ExprEval arrayExpr, ExprEval scalarExpr) { final Object[] array = arrayExpr.asArray(); - switch (scalarExpr.type()) { + switch (scalarExpr.type().getType()) { case STRING: case LONG: case DOUBLE: @@ -3416,9 +3408,9 @@ public boolean hasArrayOutput() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -3440,9 +3432,9 @@ public String name() @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -3476,7 +3468,7 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inspector, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inspector, List args) { return args.get(0).getOutputType(inspector); } @@ -3529,15 +3521,12 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) return ExprEval.of(null); } - switch (expr.type()) { + switch (expr.elementType().getType()) { case STRING: - case STRING_ARRAY: return ExprEval.ofStringArray(Arrays.copyOfRange(expr.asStringArray(), start, end)); case LONG: - case LONG_ARRAY: return ExprEval.ofLongArray(Arrays.copyOfRange(expr.asLongArray(), start, end)); case DOUBLE: - case DOUBLE_ARRAY: return ExprEval.ofDoubleArray(Arrays.copyOfRange(expr.asDoubleArray(), start, end)); } throw new RE("Unable to slice to unknown type %s", expr.type()); @@ -3560,7 +3549,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) * only LONG and DOUBLE are allowed * For a DOUBLE, it will be cast to LONG before format */ - if (valueParam.value() != null && valueParam.type() != ExprType.LONG && valueParam.type() != ExprType.DOUBLE) { + if (valueParam.value() != null && !valueParam.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) { throw new IAE("Function[%s] needs a number as its first argument", name()); } @@ -3570,7 +3559,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) long precision = 2; if (args.size() > 1) { ExprEval precisionParam = args.get(1).eval(bindings); - if (precisionParam.type() != ExprType.LONG) { + if (!precisionParam.type().is(ExprType.LONG)) { throw new IAE("Function[%s] needs an integer as its second argument", name()); } precision = precisionParam.asLong(); @@ -3592,9 +3581,9 @@ public void validateArguments(List args) @Nullable @Override - public ExprType getOutputType(Expr.InputBindingInspector inputTypes, List args) + public ExpressionType getOutputType(Expr.InputBindingInspector inputTypes, List args) { - return ExprType.STRING; + return ExpressionType.STRING; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java b/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java index eb040df04efb..d2a0b42b65ca 100644 --- a/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java @@ -125,7 +125,7 @@ public BindingAnalysis analyzeInputs() } @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return expr.getOutputType(inspector); } @@ -223,7 +223,7 @@ public BindingAnalysis analyzeInputs() } @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return function.getOutputType(inspector, args); } @@ -346,7 +346,7 @@ public BindingAnalysis analyzeInputs() @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return function.getOutputType(inspector, lambdaExpr, argsExpr); } diff --git a/core/src/main/java/org/apache/druid/math/expr/IdentifierExpr.java b/core/src/main/java/org/apache/druid/math/expr/IdentifierExpr.java index a1423c1590b1..4abfd55cad51 100644 --- a/core/src/main/java/org/apache/druid/math/expr/IdentifierExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/IdentifierExpr.java @@ -114,7 +114,7 @@ public BindingAnalysis analyzeInputs() } @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return inspector.getType(binding); } @@ -147,12 +147,12 @@ public boolean canVectorize(InputBindingInspector inspector) @Override public ExprVectorProcessor buildVectorized(VectorInputBindingInspector inspector) { - ExprType inputType = inspector.getType(binding); + ExpressionType inputType = inspector.getType(binding); if (inputType == null) { // nil column, we can be anything, so be a string because it's the most flexible // (numbers will be populated with default values in default mode and non-null) - return new IdentifierVectorProcessor(ExprType.STRING) + return new IdentifierVectorProcessor(ExpressionType.STRING) { @Override public ExprEvalVector evalVector(VectorInputBinding bindings) @@ -164,9 +164,9 @@ public ExprEvalVector evalVector(VectorInputBinding bindings) } }; } - switch (inputType) { + switch (inputType.getType()) { case LONG: - return new IdentifierVectorProcessor(ExprType.LONG) + return new IdentifierVectorProcessor(inputType) { @Override public ExprEvalVector evalVector(VectorInputBinding bindings) @@ -175,7 +175,7 @@ public ExprEvalVector evalVector(VectorInputBinding bindings) } }; case DOUBLE: - return new IdentifierVectorProcessor(ExprType.DOUBLE) + return new IdentifierVectorProcessor(inputType) { @Override public ExprEvalVector evalVector(VectorInputBinding bindings) @@ -184,7 +184,7 @@ public ExprEvalVector evalVector(VectorInputBinding bindings) } }; case STRING: - return new IdentifierVectorProcessor(ExprType.STRING) + return new IdentifierVectorProcessor(inputType) { @Override public ExprEvalVector evalVector(VectorInputBinding bindings) @@ -219,15 +219,15 @@ public int hashCode() abstract class IdentifierVectorProcessor implements ExprVectorProcessor { - private final ExprType outputType; + private final ExpressionType outputType; - public IdentifierVectorProcessor(ExprType outputType) + public IdentifierVectorProcessor(ExpressionType outputType) { this.outputType = outputType; } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { return outputType; } diff --git a/core/src/main/java/org/apache/druid/math/expr/InputBindings.java b/core/src/main/java/org/apache/druid/math/expr/InputBindings.java index 9862bcaaf872..5174603aa590 100644 --- a/core/src/main/java/org/apache/druid/math/expr/InputBindings.java +++ b/core/src/main/java/org/apache/druid/math/expr/InputBindings.java @@ -29,13 +29,13 @@ public class InputBindings /** * Create an {@link Expr.InputBindingInspector} backed by a map of binding identifiers to their {@link ExprType} */ - public static Expr.InputBindingInspector inspectorFromTypeMap(final Map types) + public static Expr.InputBindingInspector inspectorFromTypeMap(final Map types) { return new Expr.InputBindingInspector() { @Nullable @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { return types.get(name); } diff --git a/core/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java b/core/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java index a0d16ebc7e00..24ef4c82a294 100644 --- a/core/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java +++ b/core/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.math.expr.vector.VectorMathProcessors; +import org.apache.druid.segment.column.Types; import javax.annotation.Nullable; import java.util.Objects; @@ -66,7 +67,7 @@ public BindingAnalysis analyzeInputs() @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return expr.getOutputType(inspector); } @@ -124,10 +125,10 @@ public ExprEval eval(ObjectBinding bindings) if (NullHandling.sqlCompatible() && (ret.value() == null)) { return ExprEval.of(null); } - if (ret.type() == ExprType.LONG) { + if (ret.type().is(ExprType.LONG)) { return ExprEval.of(-ret.asLong()); } - if (ret.type() == ExprType.DOUBLE) { + if (ret.type().is(ExprType.DOUBLE)) { return ExprEval.of(-ret.asDouble()); } throw new IAE("unsupported type " + ret.type()); @@ -168,17 +169,17 @@ public ExprEval eval(ObjectBinding bindings) return ExprEval.of(null); } // conforming to other boolean-returning binary operators - ExprType retType = ret.type() == ExprType.DOUBLE ? ExprType.DOUBLE : ExprType.LONG; - return ExprEval.ofBoolean(!ret.asBoolean(), retType); + ExpressionType retType = ret.type().is(ExprType.DOUBLE) ? ExpressionType.DOUBLE : ExpressionType.LONG; + return ExprEval.ofBoolean(!ret.asBoolean(), retType.getType()); } @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - ExprType implicitCast = super.getOutputType(inspector); - if (ExprType.STRING.equals(implicitCast)) { - return ExprType.LONG; + ExpressionType implicitCast = super.getOutputType(inspector); + if (Types.is(implicitCast, ExprType.STRING)) { + return ExpressionType.LONG; } return implicitCast; } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/CastToDoubleVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/CastToDoubleVectorProcessor.java index 2cb82c7c1a72..da581d9caefb 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/CastToDoubleVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/CastToDoubleVectorProcessor.java @@ -20,7 +20,7 @@ package org.apache.druid.math.expr.vector; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; public final class CastToDoubleVectorProcessor extends CastToTypeVectorProcessor { @@ -37,8 +37,8 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/CastToLongVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/CastToLongVectorProcessor.java index 65d5812f5206..10a6d6f9814e 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/CastToLongVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/CastToLongVectorProcessor.java @@ -20,7 +20,7 @@ package org.apache.druid.math.expr.vector; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; public final class CastToLongVectorProcessor extends CastToTypeVectorProcessor { @@ -37,8 +37,8 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/CastToStringVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/CastToStringVectorProcessor.java index 7be4cadd5a6e..95031b788022 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/CastToStringVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/CastToStringVectorProcessor.java @@ -20,7 +20,7 @@ package org.apache.druid.math.expr.vector; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import java.util.Arrays; @@ -41,8 +41,8 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.STRING; + return ExpressionType.STRING; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/CastToTypeVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/CastToTypeVectorProcessor.java index 6c9e507b09c2..eff7633cfbab 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/CastToTypeVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/CastToTypeVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Exprs; public abstract class CastToTypeVectorProcessor implements ExprVectorProcessor @@ -31,13 +31,13 @@ protected CastToTypeVectorProcessor(ExprVectorProcessor delegate) this.delegate = delegate; } - public static ExprVectorProcessor cast(ExprVectorProcessor delegate, ExprType type) + public static ExprVectorProcessor cast(ExprVectorProcessor delegate, ExpressionType type) { final ExprVectorProcessor caster; - if (delegate.getOutputType() == type) { + if (delegate.getOutputType().equals(type)) { caster = delegate; } else { - switch (type) { + switch (type.getType()) { case STRING: caster = new CastToStringVectorProcessor(delegate); break; diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleInFunctionVectorProcessor.java index 2e0309271052..5e1aeb5aa305 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link UnivariateFunctionVectorProcessor} for processing (double[]) -> double[] @@ -29,15 +29,15 @@ public abstract class DoubleOutDoubleInFunctionVectorProcessor { public DoubleOutDoubleInFunctionVectorProcessor(ExprVectorProcessor processor, int maxVectorSize) { - super(CastToTypeVectorProcessor.cast(processor, ExprType.DOUBLE), maxVectorSize, new double[maxVectorSize]); + super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize, new double[maxVectorSize]); } public abstract double apply(double input); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleLongInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleLongInFunctionVectorProcessor.java index a3ff299f370d..9cd72912d5cf 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleLongInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoubleLongInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (double[], long[]) -> double[] @@ -34,8 +34,8 @@ public DoubleOutDoubleLongInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE), - CastToTypeVectorProcessor.cast(right, ExprType.LONG), + CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE), + CastToTypeVectorProcessor.cast(right, ExpressionType.LONG), maxVectorSize, new double[maxVectorSize] ); @@ -44,9 +44,9 @@ public DoubleOutDoubleLongInFunctionVectorProcessor( public abstract double apply(double left, long right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoublesInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoublesInFunctionVectorProcessor.java index f1d7741fdeb4..043c14d8d88e 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoublesInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutDoublesInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (double[], double[]) -> double[] @@ -34,8 +34,8 @@ public DoubleOutDoublesInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE), - CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE), + CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE), + CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE), maxVectorSize, new double[maxVectorSize] ); @@ -44,9 +44,9 @@ public DoubleOutDoublesInFunctionVectorProcessor( public abstract double apply(double left, double right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongDoubleInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongDoubleInFunctionVectorProcessor.java index 6270cdaebc73..a16876c81828 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongDoubleInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongDoubleInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (long[], double[]) -> double[] @@ -34,8 +34,8 @@ public DoubleOutLongDoubleInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.LONG), - CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE), + CastToTypeVectorProcessor.cast(left, ExpressionType.LONG), + CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE), maxVectorSize, new double[maxVectorSize] ); @@ -44,9 +44,9 @@ public DoubleOutLongDoubleInFunctionVectorProcessor( public abstract double apply(long left, double right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongInFunctionVectorProcessor.java index 06e7e757fd5d..17080126e823 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link UnivariateFunctionVectorProcessor} for processing (long[]) -> double[] @@ -29,15 +29,15 @@ public abstract class DoubleOutLongInFunctionVectorProcessor { public DoubleOutLongInFunctionVectorProcessor(ExprVectorProcessor processor, int maxVectorSize) { - super(CastToTypeVectorProcessor.cast(processor, ExprType.LONG), maxVectorSize, new double[maxVectorSize]); + super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize, new double[maxVectorSize]); } public abstract double apply(long input); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongsInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongsInFunctionVectorProcessor.java index b86fd9906c68..e35664811f81 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongsInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/DoubleOutLongsInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (long[], long[]) -> double[] @@ -34,8 +34,8 @@ public DoubleOutLongsInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.LONG), - CastToTypeVectorProcessor.cast(right, ExprType.LONG), + CastToTypeVectorProcessor.cast(left, ExpressionType.LONG), + CastToTypeVectorProcessor.cast(right, ExpressionType.LONG), maxVectorSize, new double[maxVectorSize] ); @@ -44,9 +44,9 @@ public DoubleOutLongsInFunctionVectorProcessor( public abstract double apply(long left, long right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalDoubleVector.java b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalDoubleVector.java index 8817c344ab25..ef13ffa9918d 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalDoubleVector.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalDoubleVector.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import java.util.Arrays; @@ -31,9 +31,9 @@ public ExprEvalDoubleVector(double[] values, boolean[] nulls) } @Override - public ExprType getType() + public ExpressionType getType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalLongVector.java b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalLongVector.java index 93f2cbab54dd..aa38b172da02 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalLongVector.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalLongVector.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; import java.util.Arrays; @@ -32,9 +32,9 @@ public ExprEvalLongVector(long[] values, @Nullable boolean[] nulls) } @Override - public ExprType getType() + public ExpressionType getType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalStringVector.java b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalStringVector.java index 304e70e5edb4..257e5ecdf801 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalStringVector.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalStringVector.java @@ -21,7 +21,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.ExprEval; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; @@ -70,9 +70,9 @@ public boolean[] getNullVector() } @Override - public ExprType getType() + public ExpressionType getType() { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalVector.java b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalVector.java index ddb607d36c34..fdb6605989e6 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalVector.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/ExprEvalVector.java @@ -20,7 +20,7 @@ package org.apache.druid.math.expr.vector; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; import java.lang.reflect.Array; @@ -63,7 +63,7 @@ public boolean[] getNullVector() return nulls; } - public abstract ExprType getType(); + public abstract ExpressionType getType(); public abstract long[] getLongVector(); diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/ExprVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/ExprVectorProcessor.java index 24688124df2d..be0e4284270b 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/ExprVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/ExprVectorProcessor.java @@ -20,7 +20,7 @@ package org.apache.druid.math.expr.vector; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * Interface describing vectorized expression processors, which can be specialized using input type information to @@ -30,5 +30,5 @@ public interface ExprVectorProcessor { ExprEvalVector evalVector(Expr.VectorInputBinding bindings); - ExprType getOutputType(); + ExpressionType getOutputType(); } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleInFunctionVectorProcessor.java index 3fedb9dba924..e85c45214f14 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link UnivariateFunctionVectorProcessor} for processing (long[]) -> long[] @@ -28,15 +28,15 @@ public abstract class LongOutDoubleInFunctionVectorProcessor extends UnivariateF { public LongOutDoubleInFunctionVectorProcessor(ExprVectorProcessor processor, int maxVectorSize) { - super(CastToTypeVectorProcessor.cast(processor, ExprType.DOUBLE), maxVectorSize, new long[maxVectorSize]); + super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize, new long[maxVectorSize]); } public abstract long apply(double input); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleLongInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleLongInFunctionVectorProcessor.java index 4f66b35acfbe..c45d76b7d39b 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleLongInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoubleLongInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (double[], long[]) -> long[] @@ -34,8 +34,8 @@ public LongOutDoubleLongInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE), - CastToTypeVectorProcessor.cast(right, ExprType.LONG), + CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE), + CastToTypeVectorProcessor.cast(right, ExpressionType.LONG), maxVectorSize, new long[maxVectorSize] ); @@ -44,9 +44,9 @@ public LongOutDoubleLongInFunctionVectorProcessor( public abstract long apply(double left, long right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoublesInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoublesInFunctionVectorProcessor.java index f1aa5af9a070..483076660525 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoublesInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutDoublesInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (double[], double[]) -> long[] @@ -34,8 +34,8 @@ public LongOutDoublesInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.DOUBLE), - CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE), + CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE), + CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE), maxVectorSize, new long[maxVectorSize] ); @@ -44,9 +44,9 @@ public LongOutDoublesInFunctionVectorProcessor( public abstract long apply(double left, double right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongDoubleInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongDoubleInFunctionVectorProcessor.java index 8476619f02d6..4a9e7aa2e9d0 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongDoubleInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongDoubleInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (long[], double[]) -> long[] @@ -34,8 +34,8 @@ public LongOutLongDoubleInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.LONG), - CastToTypeVectorProcessor.cast(right, ExprType.DOUBLE), + CastToTypeVectorProcessor.cast(left, ExpressionType.LONG), + CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE), maxVectorSize, new long[maxVectorSize] ); @@ -44,9 +44,9 @@ public LongOutLongDoubleInFunctionVectorProcessor( public abstract long apply(long left, double right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongInFunctionVectorProcessor.java index c3cc20e62678..1b728b538684 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link UnivariateFunctionVectorProcessor} for processing (long[]) -> long[] @@ -28,15 +28,15 @@ public abstract class LongOutLongInFunctionVectorProcessor extends UnivariateFun { public LongOutLongInFunctionVectorProcessor(ExprVectorProcessor processor, int maxVectorSize) { - super(CastToTypeVectorProcessor.cast(processor, ExprType.LONG), maxVectorSize, new long[maxVectorSize]); + super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize, new long[maxVectorSize]); } public abstract long apply(long input); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongsInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongsInFunctionVectorProcessor.java index 5fa3735a8347..ebbb6587505c 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongsInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutLongsInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link BivariateFunctionVectorProcessor} for processing (long[], long[]) -> long[] @@ -34,8 +34,8 @@ public LongOutLongsInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.LONG), - CastToTypeVectorProcessor.cast(right, ExprType.LONG), + CastToTypeVectorProcessor.cast(left, ExpressionType.LONG), + CastToTypeVectorProcessor.cast(right, ExpressionType.LONG), maxVectorSize, new long[maxVectorSize] ); @@ -44,9 +44,9 @@ public LongOutLongsInFunctionVectorProcessor( public abstract long apply(long left, long right); @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringInFunctionVectorProcessor.java index 496ed987e655..2ebc51908616 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * specialized {@link UnivariateFunctionVectorObjectProcessor} for processing (String[]) -> long[] @@ -29,13 +29,13 @@ public abstract class LongOutStringInFunctionVectorProcessor { public LongOutStringInFunctionVectorProcessor(ExprVectorProcessor processor, int maxVectorSize) { - super(CastToTypeVectorProcessor.cast(processor, ExprType.STRING), maxVectorSize, new long[maxVectorSize]); + super(CastToTypeVectorProcessor.cast(processor, ExpressionType.STRING), maxVectorSize, new long[maxVectorSize]); } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringsInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringsInFunctionVectorProcessor.java index e7ddb6d5e99a..d450b8633045 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringsInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/LongOutStringsInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; @@ -34,8 +34,8 @@ protected LongOutStringsInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.STRING), - CastToTypeVectorProcessor.cast(right, ExprType.STRING), + CastToTypeVectorProcessor.cast(left, ExpressionType.STRING), + CastToTypeVectorProcessor.cast(right, ExpressionType.STRING), maxVectorSize, new long[maxVectorSize] ); @@ -72,8 +72,8 @@ ExprEvalVector asEval() } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/StringOutMultiStringInVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/StringOutMultiStringInVectorProcessor.java index 8c6844551c23..1008098d6051 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/StringOutMultiStringInVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/StringOutMultiStringInVectorProcessor.java @@ -21,7 +21,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; /** * many strings enter, one string leaves... @@ -44,9 +44,9 @@ protected StringOutMultiStringInVectorProcessor( } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/StringOutStringsInFunctionVectorProcessor.java b/core/src/main/java/org/apache/druid/math/expr/vector/StringOutStringsInFunctionVectorProcessor.java index 35f38bc09d6c..7bd7ed127722 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/StringOutStringsInFunctionVectorProcessor.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/StringOutStringsInFunctionVectorProcessor.java @@ -19,7 +19,7 @@ package org.apache.druid.math.expr.vector; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; @@ -33,8 +33,8 @@ protected StringOutStringsInFunctionVectorProcessor( ) { super( - CastToTypeVectorProcessor.cast(left, ExprType.STRING), - CastToTypeVectorProcessor.cast(right, ExprType.STRING), + CastToTypeVectorProcessor.cast(left, ExpressionType.STRING), + CastToTypeVectorProcessor.cast(right, ExpressionType.STRING), maxVectorSize, new String[maxVectorSize] ); @@ -62,8 +62,8 @@ ExprEvalVector asEval() } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.STRING; + return ExpressionType.STRING; } } diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/VectorComparisonProcessors.java b/core/src/main/java/org/apache/druid/math/expr/vector/VectorComparisonProcessors.java index 712b6b924643..23332248bdbe 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/VectorComparisonProcessors.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/VectorComparisonProcessors.java @@ -23,6 +23,8 @@ import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.segment.column.Types; import javax.annotation.Nullable; import java.util.Objects; @@ -41,20 +43,20 @@ public static ExprVectorProcessor makeComparisonProcessor( Supplier doubleOutDoublesInProcessor ) { - final ExprType leftType = left.getOutputType(inspector); - final ExprType rightType = right.getOutputType(inspector); + final ExpressionType leftType = left.getOutputType(inspector); + final ExpressionType rightType = right.getOutputType(inspector); ExprVectorProcessor processor = null; - if (leftType == ExprType.STRING) { - if (rightType == null || rightType == ExprType.STRING) { + if (Types.is(leftType, ExprType.STRING)) { + if (Types.isNullOr(rightType, ExprType.STRING)) { processor = longOutStringsInFunctionVectorProcessor.get(); } else { processor = doubleOutDoublesInProcessor.get(); } } else if (leftType == null) { - if (rightType == ExprType.STRING || rightType == null) { + if (Types.isNullOr(rightType, ExprType.STRING)) { processor = longOutStringsInFunctionVectorProcessor.get(); } - } else if (leftType == ExprType.DOUBLE || rightType == ExprType.DOUBLE) { + } else if (leftType.is(ExprType.DOUBLE) || Types.is(rightType, ExprType.DOUBLE)) { processor = doubleOutDoublesInProcessor.get(); } if (processor != null) { diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/VectorMathProcessors.java b/core/src/main/java/org/apache/druid/math/expr/vector/VectorMathProcessors.java index 0737f252f8b4..d93274a3a461 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/VectorMathProcessors.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/VectorMathProcessors.java @@ -24,7 +24,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Exprs; +import org.apache.druid.segment.column.Types; import java.util.function.Supplier; @@ -42,13 +44,15 @@ public static ExprVectorProcessor makeMathProcessor( Supplier doubleOutDoubleInSupplier ) { - final ExprType inputType = arg.getOutputType(inspector); + final ExpressionType inputType = arg.getOutputType(inspector); ExprVectorProcessor processor = null; - if (inputType == ExprType.LONG) { - processor = longOutLongInSupplier.get(); - } else if (inputType == ExprType.DOUBLE) { - processor = doubleOutDoubleInSupplier.get(); + if (inputType != null) { + if (inputType.is(ExprType.LONG)) { + processor = longOutLongInSupplier.get(); + } else if (inputType.is(ExprType.DOUBLE)) { + processor = doubleOutDoubleInSupplier.get(); + } } if (processor == null) { throw Exprs.cannotVectorize(); @@ -68,13 +72,15 @@ public static ExprVectorProcessor makeDoubleMathProcessor( Supplier doubleOutDoubleInSupplier ) { - final ExprType inputType = arg.getOutputType(inspector); + final ExpressionType inputType = arg.getOutputType(inspector); ExprVectorProcessor processor = null; - if (inputType == ExprType.LONG) { - processor = doubleOutLongInSupplier.get(); - } else if (inputType == ExprType.DOUBLE) { - processor = doubleOutDoubleInSupplier.get(); + if (inputType != null) { + if (inputType.is(ExprType.LONG)) { + processor = doubleOutLongInSupplier.get(); + } else if (inputType.is(ExprType.DOUBLE)) { + processor = doubleOutDoubleInSupplier.get(); + } } if (processor == null) { throw Exprs.cannotVectorize(); @@ -94,13 +100,15 @@ public static ExprVectorProcessor makeLongMathProcessor( Supplier longOutDoubleInSupplier ) { - final ExprType inputType = arg.getOutputType(inspector); + final ExpressionType inputType = arg.getOutputType(inspector); ExprVectorProcessor processor = null; - if (inputType == ExprType.LONG) { - processor = longOutLongInSupplier.get(); - } else if (inputType == ExprType.DOUBLE) { - processor = longOutDoubleInSupplier.get(); + if (inputType != null) { + if (inputType.is(ExprType.LONG)) { + processor = longOutLongInSupplier.get(); + } else if (inputType.is(ExprType.DOUBLE)) { + processor = longOutDoubleInSupplier.get(); + } } if (processor == null) { throw Exprs.cannotVectorize(); @@ -125,33 +133,33 @@ public static ExprVectorProcessor makeMathProcessor( Supplier doubleOutDoublesInProcessor ) { - final ExprType leftType = left.getOutputType(inspector); - final ExprType rightType = right.getOutputType(inspector); + final ExpressionType leftType = left.getOutputType(inspector); + final ExpressionType rightType = right.getOutputType(inspector); ExprVectorProcessor processor = null; - if (leftType == ExprType.LONG) { - if (rightType == null || rightType == ExprType.LONG) { + if (Types.is(leftType, ExprType.LONG)) { + if (Types.isNullOr(rightType, ExprType.LONG)) { processor = longOutLongsInProcessor.get(); - } else if (rightType == ExprType.STRING || rightType == ExprType.DOUBLE) { + } else if (rightType.anyOf(ExprType.STRING, ExprType.DOUBLE)) { processor = doubleOutLongDoubleInProcessor.get(); } - } else if (leftType == ExprType.DOUBLE) { - if (rightType == ExprType.LONG) { + } else if (Types.is(leftType, ExprType.DOUBLE)) { + if (Types.is(rightType, ExprType.LONG)) { processor = doubleOutDoubleLongInProcessor.get(); - } else if (rightType == null || rightType == ExprType.STRING || rightType == ExprType.DOUBLE) { + } else if (Types.isNullOrAnyOf(rightType, ExprType.STRING, ExprType.DOUBLE)) { processor = doubleOutDoublesInProcessor.get(); } } else if (leftType == null) { - if (rightType == ExprType.LONG) { + if (Types.is(rightType, ExprType.LONG)) { processor = longOutLongsInProcessor.get(); - } else if (rightType == ExprType.DOUBLE) { + } else if (Types.is(rightType, ExprType.DOUBLE)) { processor = doubleOutLongDoubleInProcessor.get(); } else if (rightType == null) { processor = longOutLongsInProcessor.get(); } - } else if (leftType == ExprType.STRING) { - if (rightType == ExprType.LONG) { + } else if (leftType.is(ExprType.STRING)) { + if (Types.is(rightType, ExprType.LONG)) { processor = longOutLongsInProcessor.get(); - } else if (rightType == ExprType.DOUBLE) { + } else if (Types.is(rightType, ExprType.DOUBLE)) { processor = doubleOutLongDoubleInProcessor.get(); } } @@ -178,25 +186,25 @@ public static ExprVectorProcessor makeDoubleMathProcessor( Supplier doubleOutDoublesInProcessor ) { - final ExprType leftType = left.getOutputType(inspector); - final ExprType rightType = right.getOutputType(inspector); + final ExpressionType leftType = left.getOutputType(inspector); + final ExpressionType rightType = right.getOutputType(inspector); ExprVectorProcessor processor = null; - if (leftType == ExprType.LONG) { - if (rightType == ExprType.LONG) { + if (Types.is(leftType, ExprType.LONG)) { + if (Types.is(rightType, ExprType.LONG)) { processor = doubleOutLongsInProcessor.get(); - } else if (rightType == null || rightType == ExprType.DOUBLE) { + } else if (Types.isNullOr(rightType, ExprType.DOUBLE)) { processor = doubleOutLongDoubleInProcessor.get(); } - } else if (leftType == ExprType.DOUBLE) { - if (rightType == ExprType.LONG) { + } else if (Types.is(leftType, ExprType.DOUBLE)) { + if (Types.is(rightType, ExprType.LONG)) { processor = doubleOutDoubleLongInProcessor.get(); - } else if (rightType == null || rightType == ExprType.DOUBLE) { + } else if (Types.isNullOr(rightType, ExprType.DOUBLE)) { processor = doubleOutDoublesInProcessor.get(); } } else if (leftType == null) { - if (rightType == ExprType.LONG) { + if (Types.is(rightType, ExprType.LONG)) { processor = doubleOutDoubleLongInProcessor.get(); - } else if (rightType == ExprType.DOUBLE) { + } else if (Types.is(rightType, ExprType.DOUBLE)) { processor = doubleOutDoublesInProcessor.get(); } } @@ -223,25 +231,25 @@ public static ExprVectorProcessor makeLongMathProcessor( Supplier longOutDoublesInProcessor ) { - final ExprType leftType = left.getOutputType(inspector); - final ExprType rightType = right.getOutputType(inspector); + final ExpressionType leftType = left.getOutputType(inspector); + final ExpressionType rightType = right.getOutputType(inspector); ExprVectorProcessor processor = null; - if (leftType == ExprType.LONG) { - if (rightType == null || rightType == ExprType.LONG) { + if (Types.is(leftType, ExprType.LONG)) { + if (Types.isNullOr(rightType, ExprType.LONG)) { processor = longOutLongsInProcessor.get(); - } else if (rightType == ExprType.DOUBLE) { + } else if (rightType.is(ExprType.DOUBLE)) { processor = longOutLongDoubleInProcessor.get(); } - } else if (leftType == ExprType.DOUBLE) { - if (rightType == ExprType.LONG) { + } else if (Types.is(leftType, ExprType.DOUBLE)) { + if (Types.is(rightType, ExprType.LONG)) { processor = longOutDoubleLongInProcessor.get(); - } else if (rightType == null || rightType == ExprType.DOUBLE) { + } else if (Types.isNullOr(rightType, ExprType.DOUBLE)) { processor = longOutDoublesInProcessor.get(); } } else if (leftType == null) { - if (rightType == ExprType.LONG) { + if (Types.is(rightType, ExprType.LONG)) { processor = longOutLongsInProcessor.get(); - } else if (rightType == ExprType.DOUBLE) { + } else if (Types.is(rightType, ExprType.DOUBLE)) { processor = longOutDoublesInProcessor.get(); } } @@ -686,11 +694,11 @@ public static ExprVectorProcessor doublePower( Expr right ) { - final ExprType leftType = left.getOutputType(inspector); - final ExprType rightType = right.getOutputType(inspector); + final ExpressionType leftType = left.getOutputType(inspector); + final ExpressionType rightType = right.getOutputType(inspector); BivariateFunctionVectorProcessor processor = null; - if ((leftType == ExprType.LONG && (rightType == null || rightType == ExprType.LONG)) || - (leftType == null && rightType == ExprType.LONG)) { + if ((Types.is(leftType, ExprType.LONG) && Types.isNullOr(rightType, ExprType.LONG)) || + (leftType == null && Types.is(rightType, ExprType.LONG))) { processor = new DoubleOutLongsInFunctionVectorProcessor( left.buildVectorized(inspector), right.buildVectorized(inspector), @@ -2012,10 +2020,10 @@ public static ExprVectorProcessor bitwiseConvertDoubleToLongBits( Expr arg ) { - final ExprType inputType = arg.getOutputType(inspector); + final ExpressionType inputType = arg.getOutputType(inspector); ExprVectorProcessor processor = null; - if (inputType == ExprType.LONG) { + if (Types.is(inputType, ExprType.LONG)) { processor = new LongOutLongInFunctionVectorProcessor( arg.buildVectorized(inspector), inspector.getMaxVectorSize() @@ -2027,7 +2035,7 @@ public long apply(long input) return Double.doubleToLongBits(input); } }; - } else if (inputType == ExprType.DOUBLE) { + } else if (Types.is(inputType, ExprType.DOUBLE)) { processor = new LongOutDoubleInFunctionVectorProcessor( arg.buildVectorized(inspector), inspector.getMaxVectorSize() @@ -2051,10 +2059,10 @@ public static ExprVectorProcessor bitwiseConvertLongBitsToDouble( Expr arg ) { - final ExprType inputType = arg.getOutputType(inspector); + final ExpressionType inputType = arg.getOutputType(inspector); ExprVectorProcessor processor = null; - if (inputType == ExprType.LONG) { + if (Types.is(inputType, ExprType.LONG)) { processor = new DoubleOutLongInFunctionVectorProcessor( arg.buildVectorized(inspector), inspector.getMaxVectorSize() @@ -2066,7 +2074,7 @@ public double apply(long input) return Double.longBitsToDouble(input); } }; - } else if (inputType == ExprType.DOUBLE) { + } else if (Types.is(inputType, ExprType.DOUBLE)) { processor = new DoubleOutDoubleInFunctionVectorProcessor( arg.buildVectorized(inspector), inspector.getMaxVectorSize() diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java b/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java index 60edbdb44ed4..e8815f4a6826 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java @@ -21,7 +21,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; import java.util.Arrays; @@ -42,9 +42,9 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.STRING; + return ExpressionType.STRING; } }; } @@ -70,9 +70,9 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; } }; } @@ -98,9 +98,9 @@ public ExprEvalVector evalVector(Expr.VectorInputBinding bindings) } @Override - public ExprType getOutputType() + public ExpressionType getOutputType() { - return ExprType.LONG; + return ExpressionType.LONG; } }; } @@ -108,7 +108,7 @@ public ExprType getOutputType() public static ExprVectorProcessor parseLong(Expr.VectorInputBindingInspector inspector, Expr arg, int radix) { final ExprVectorProcessor processor = new LongOutStringInFunctionVectorProcessor( - CastToTypeVectorProcessor.cast(arg.buildVectorized(inspector), ExprType.STRING), + CastToTypeVectorProcessor.cast(arg.buildVectorized(inspector), ExpressionType.STRING), inspector.getMaxVectorSize() ) { diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/VectorStringProcessors.java b/core/src/main/java/org/apache/druid/math/expr/vector/VectorStringProcessors.java index bbfbd68634ab..35c966a75465 100644 --- a/core/src/main/java/org/apache/druid/math/expr/vector/VectorStringProcessors.java +++ b/core/src/main/java/org/apache/druid/math/expr/vector/VectorStringProcessors.java @@ -21,7 +21,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nullable; import java.util.List; @@ -68,7 +68,7 @@ public static ExprVectorProcessor concat(Expr.VectorInputBindingInspector { final ExprVectorProcessor[] inputProcessors = new ExprVectorProcessor[inputs.size()]; for (int i = 0; i < inputs.size(); i++) { - inputProcessors[i] = CastToTypeVectorProcessor.cast(inputs.get(i).buildVectorized(inspector), ExprType.STRING); + inputProcessors[i] = CastToTypeVectorProcessor.cast(inputs.get(i).buildVectorized(inspector), ExpressionType.STRING); } final ExprVectorProcessor processor = new StringOutMultiStringInVectorProcessor( inputProcessors, diff --git a/core/src/main/java/org/apache/druid/segment/column/BaseTypeSignature.java b/core/src/main/java/org/apache/druid/segment/column/BaseTypeSignature.java new file mode 100644 index 000000000000..819883cec87f --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/BaseTypeSignature.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.column; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Objects; + +public abstract class BaseTypeSignature implements TypeSignature +{ + protected final Type type; + + @Nullable + protected final String complexTypeName; + + @Nullable + protected final TypeSignature elementType; + + public BaseTypeSignature( + Type type, + @Nullable String complexTypeName, + @Nullable TypeSignature elementType + ) + { + this.type = type; + this.complexTypeName = complexTypeName; + this.elementType = elementType; + } + + @Override + @JsonProperty("type") + public Type getType() + { + return type; + } + + @Override + @Nullable + @JsonProperty("complexTypeName") + public String getComplexTypeName() + { + return complexTypeName; + } + + @Override + @Nullable + @JsonProperty("elementType") + public TypeSignature getElementType() + { + return elementType; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseTypeSignature that = (BaseTypeSignature) o; + return type.equals(that.type) + && Objects.equals(complexTypeName, that.complexTypeName) + && Objects.equals(elementType, that.elementType); + } + + @Override + public int hashCode() + { + return Objects.hash(type, complexTypeName, elementType); + } + + @Override + public String toString() + { + // this is also used for JSON serialization by the sub-classes, but class is decorated with JSON annotations + // to leave the option open someday of switching to standard serialization + return asTypeString(); + } +} diff --git a/core/src/main/java/org/apache/druid/segment/column/ColumnType.java b/core/src/main/java/org/apache/druid/segment/column/ColumnType.java new file mode 100644 index 000000000000..31b673ad2143 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/ColumnType.java @@ -0,0 +1,68 @@ +/* + * 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.column; + +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 javax.annotation.Nullable; + +@JsonSerialize(using = ToStringSerializer.class) +public class ColumnType extends BaseTypeSignature +{ + public static final ColumnType STRING = new ColumnType(ValueType.STRING, null, null); + public static final ColumnType LONG = new ColumnType(ValueType.LONG, null, null); + public static final ColumnType DOUBLE = new ColumnType(ValueType.DOUBLE, null, null); + public static final ColumnType FLOAT = new ColumnType(ValueType.FLOAT, null, null); + // currently, arrays only come from expressions or aggregators + // and there are no native float expressions (or aggs which produce float arrays) + public static final ColumnType STRING_ARRAY = new ColumnType(ValueType.ARRAY, null, STRING); + public static final ColumnType LONG_ARRAY = new ColumnType(ValueType.ARRAY, null, LONG); + public static final ColumnType DOUBLE_ARRAY = new ColumnType(ValueType.ARRAY, null, DOUBLE); + public static final ColumnType UNKNOWN_COMPLEX = new ColumnType(ValueType.COMPLEX, null, null); + + @JsonCreator + public ColumnType( + @JsonProperty("type") ValueType type, + @JsonProperty("complexTypeName") @Nullable String complexTypeName, + @JsonProperty("elementType") @Nullable ColumnType elementType + ) + { + super(type, complexTypeName, elementType); + } + + @Nullable + @JsonCreator + public static ColumnType fromString(@Nullable String typeName) + { + return Types.fromString(ColumnTypeFactory.getInstance(), typeName); + } + + public static ColumnType ofArray(ColumnType elementType) + { + return ColumnTypeFactory.getInstance().ofArray(elementType); + } + public static ColumnType ofComplex(String complexTypeName) + { + return ColumnTypeFactory.getInstance().ofComplex(complexTypeName); + } +} diff --git a/core/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java b/core/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java new file mode 100644 index 000000000000..ca34af3c8977 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java @@ -0,0 +1,140 @@ +/* + * 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.column; + +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; + +public class ColumnTypeFactory implements TypeFactory +{ + private static final ColumnTypeFactory INSTANCE = new ColumnTypeFactory(); + + private static final Interner INTERNER = Interners.newWeakInterner(); + + public static ColumnTypeFactory getInstance() + { + return INSTANCE; + } + + private ColumnTypeFactory() + { + // no instantiation + } + + public static ColumnType ofType(TypeSignature type) + { + switch (type.getType()) { + case LONG: + return ColumnType.LONG; + case FLOAT: + return ColumnType.FLOAT; + case DOUBLE: + return ColumnType.DOUBLE; + case STRING: + return ColumnType.STRING; + case ARRAY: + switch (type.getElementType().getType()) { + case LONG: + return ColumnType.LONG_ARRAY; + case DOUBLE: + return ColumnType.DOUBLE_ARRAY; + case STRING: + return ColumnType.STRING_ARRAY; + default: + throw new ISE("Unsupported expression type[%s]", type.asTypeString()); + } + case COMPLEX: + return INTERNER.intern(new ColumnType(ValueType.COMPLEX, type.getComplexTypeName(), null)); + default: + throw new ISE("Unsupported column type[%s]", type.asTypeString()); + } + } + + public static ColumnType ofValueType(ValueType type) + { + switch (type) { + case LONG: + return ColumnType.LONG; + case FLOAT: + return ColumnType.FLOAT; + case DOUBLE: + return ColumnType.DOUBLE; + case STRING: + return ColumnType.STRING; + case COMPLEX: + return ColumnType.UNKNOWN_COMPLEX; + default: + throw new ISE("Unsupported column type[%s]", type); + } + } + + @Override + public ColumnType ofString() + { + return ColumnType.STRING; + } + + @Override + public ColumnType ofFloat() + { + return ColumnType.FLOAT; + } + + @Override + public ColumnType ofDouble() + { + return ColumnType.DOUBLE; + } + + @Override + public ColumnType ofLong() + { + return ColumnType.LONG; + } + + @Override + public ColumnType ofArray(ColumnType elementType) + { + if (elementType.isPrimitive()) { + switch (elementType.getType()) { + case STRING: + return ColumnType.STRING_ARRAY; + case DOUBLE: + return ColumnType.DOUBLE_ARRAY; + case LONG: + return ColumnType.LONG_ARRAY; + } + } + // i guess this is potentially unbounded if we ever support arbitrarily deep nested arrays + return INTERNER.intern(new ColumnType(ValueType.ARRAY, null, elementType)); + } + + @Override + public ColumnType ofComplex(@Nullable String complexTypeName) + { + if (complexTypeName == null) { + return ColumnType.UNKNOWN_COMPLEX; + } + return INTERNER.intern(new ColumnType(ValueType.COMPLEX, complexTypeName, null)); + } +} diff --git a/core/src/main/java/org/apache/druid/segment/column/TypeDescriptor.java b/core/src/main/java/org/apache/druid/segment/column/TypeDescriptor.java new file mode 100644 index 000000000000..5826493fae27 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/TypeDescriptor.java @@ -0,0 +1,63 @@ +/* + * 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.column; + + +/** + * This is a bridge between {@link ValueType} and {@link org.apache.druid.math.expr.ExprType}, so that they can both + * be used with {@link TypeSignature}. + * + * This interface assists in classifying the specific enum values that implement this interface into the current type + * families: 'primitives', 'arrays', and 'complex' types through {@link #isPrimitive()} and {@link #isArray()} + * + * At some point we should probably consider reworking this into a 'TypeFamily' enum that maps these high level families + * to specific types. + * + * This interface can potentially be removed if the expression processing system is updated to use {@link ColumnType} + * instead of {@link org.apache.druid.math.expr.ExpressionType}, which would allow + * {@link org.apache.druid.math.expr.ExprType} to be removed and this interface merged into {@link ValueType} (along + * with consolidation of several other interfaces, see {@link TypeSignature} javadoc for additional details). + */ +public interface TypeDescriptor +{ + /** + * Scalar numeric primitive values. + * + * @see ValueType#isNumeric + * @see org.apache.druid.math.expr.ExprType#isNumeric + */ + boolean isNumeric(); + + /** + * Scalar numeric and string values. This does not currently include complex types. + * + * @see ValueType#isPrimitive + * @see org.apache.druid.math.expr.ExprType#isPrimitive + */ + boolean isPrimitive(); + + /** + * Value is an array of some other {@link TypeDescriptor} + * + * @see ValueType#isArray + * @see org.apache.druid.math.expr.ExprType#isArray + */ + boolean isArray(); +} diff --git a/core/src/main/java/org/apache/druid/segment/column/TypeFactory.java b/core/src/main/java/org/apache/druid/segment/column/TypeFactory.java new file mode 100644 index 000000000000..cba76dbe8ad2 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/TypeFactory.java @@ -0,0 +1,37 @@ +/* + * 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.column; + +import javax.annotation.Nullable; + +/** + * Create {@link TypeSignature} of a {@link TypeDescriptor}. Useful for creating types from + * {@link TypeSignature#asTypeString()}} or converting between {@link TypeSignature} of different {@link TypeDescriptor} + * implementations. Implementations also offer object interning for arbitrary array and complex types. + */ +public interface TypeFactory> +{ + T ofString(); + T ofFloat(); + T ofDouble(); + T ofLong(); + T ofArray(T elementType); + T ofComplex(@Nullable String complexTypeName); +} diff --git a/core/src/main/java/org/apache/druid/segment/column/TypeSignature.java b/core/src/main/java/org/apache/druid/segment/column/TypeSignature.java new file mode 100644 index 000000000000..acf5eef4a505 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/TypeSignature.java @@ -0,0 +1,164 @@ +/* + * 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.column; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * This interface serves as a common foundation for Druids native type system, and provides common methods for reasoning + * about and handling type matters. Additional type common type handling methods are provided by {@link Types} utility. + * + * This information is used by Druid to make decisions about how to correctly process inputs and determine output types + * at all layers of the engine, from how to group, filter, aggregate, and transform columns up to how to best plan SQL + * into native Druid queries. + * + * The native Druid type system can currently be broken down at a high level into 'primitive' types, 'array' types, and + * 'complex' types, and this classification is defined by an enumeration which implements {@link TypeDescriptor} such + * as {@link ValueType} for the general query engines and {@link org.apache.druid.math.expr.ExprType} for low level + * expression processing. This is exposed via {@link #getType()}, and will be most callers first point of contact with + * the {@link TypeSignature} when trying to decide how to handle a given input. + * + * Druid 'primitive' types includes strings and numeric types. Note: multi-value string columns are still considered + * 'primitive' string types, because they do not behave as traditional arrays (unless explicitly converted to an array), + * and are always serialized as opportunistically single valued, so whether or not any particular string column is + * multi-valued might vary from segment to segment. The concept of multi-valued strings only exists at a very low + * engine level and are only modeled by the ColumnCapabilities implementation of {@link TypeSignature}. + * + * 'array' types contain additional nested type information about the elements of an array, a reference to another + * {@link TypeSignature} through the {@link #getElementType()} method. If {@link TypeDescriptor#isArray()} is true, + * then {@link #getElementType()} should never return null. + * + * 'complex' types are Druids extensible types, which have a registry that allows these types to be defined and + * associated with a name which is available as {@link #getComplexTypeName()}. These type names are unique, so this + * information is used to allow handling of these 'complex' types to confirm. + * + * {@link TypeSignature} is currently manifested in 3 forms: {@link ColumnType} which is the high level 'native' Druid + * type definitions using {@link ValueType}, and is used by row signatures and SQL schemas, used by callers as input + * to various API methods, and most general purpose type handling. In 'druid-processing' there is an additional + * type ... type, ColumnCapabilities, which is effectively a {@link ColumnType} but includes some additional + * information for low level query processing, such as details about whether a column has indexes, dictionaries, null + * values, is a multi-value string column, and more. + * + * The third is {@link org.apache.druid.math.expr.ExpressionType}, which instead of {@link ValueType} uses + * {@link org.apache.druid.math.expr.ExprType}, and is used exclusively for handling Druid native expression evaluation. + * {@link org.apache.druid.math.expr.ExpressionType} exists because the Druid expression system does not natively + * handle float types, so it is essentially a mapping of {@link ColumnType} where floats are coerced to double typed + * values. Ideally at some point Druid expressions can just handle floats directly, and these two {@link TypeSignature} + * can be merged, which will simplify this interface to no longer need be generic, allow {@link ColumnType} to be + * collapsed into {@link BaseTypeSignature}, and finally unify the type system. + */ +public interface TypeSignature +{ + /** + * {@link TypeDescriptor} enumeration used to handle different classes of types + * + * @see ValueType + * @see org.apache.druid.math.expr.ExprType + */ + Type getType(); + + /** + * Type name of 'complex' types ({@link ValueType#COMPLEX}, {@link org.apache.druid.math.expr.ExprType#COMPLEX}), + * which are 'registered' by their name, acting as a key to get the correct set of serialization, deserialization, + * and other type specific handling facilties. + * + * For other types, this value will be null. + */ + @Nullable + String getComplexTypeName(); + + /** + * {@link TypeSignature} for the elements contained in an array type ({@link ValueType#ARRAY}, + * {@link org.apache.druid.math.expr.ExprType#ARRAY}). + * + * For non-array types, this value will be null. + */ + @Nullable + TypeSignature getElementType(); + + /** + * Check if the value of {@link #getType()} is equal to the candidate {@link TypeDescriptor}. + */ + default boolean is(Type candidate) + { + return Objects.equals(getType(), candidate); + } + + /** + * Check if the value of {@link #getType()} matches any of the {@link TypeDescriptor} specified. + */ + default boolean anyOf(Type... types) + { + for (Type candidate : types) { + if (Objects.equals(getType(), candidate)) { + return true; + } + } + return false; + } + + /** + * Check if the type is numeric ({@link TypeDescriptor#isNumeric()}) + */ + @JsonIgnore + default boolean isNumeric() + { + return getType().isNumeric(); + } + + /** + * Check if the type is a primitive ({@link TypeDescriptor#isPrimitive()}, e.g. not an array, not a complex type) + */ + @JsonIgnore + default boolean isPrimitive() + { + return getType().isPrimitive(); + } + + /** + * Check if the type is an array ({@link TypeDescriptor#isArray()}) + */ + @JsonIgnore + default boolean isArray() + { + return getType().isArray(); + } + + /** + * Convert a {@link TypeSignature} into a simple string. This value can be converted back into a {@link TypeSignature} + * with {@link Types#fromString(TypeFactory, String)}. + */ + @JsonIgnore + default String asTypeString() + { + if (isArray()) { + return StringUtils.format("ARRAY<%s>", getElementType()); + } + final String complexTypeName = getComplexTypeName(); + if (!isPrimitive()) { + return complexTypeName == null ? "COMPLEX" : StringUtils.format("COMPLEX<%s>", complexTypeName); + } + return getType().toString(); + } +} diff --git a/core/src/main/java/org/apache/druid/segment/column/Types.java b/core/src/main/java/org/apache/druid/segment/column/Types.java new file mode 100644 index 000000000000..44c7b13720e7 --- /dev/null +++ b/core/src/main/java/org/apache/druid/segment/column/Types.java @@ -0,0 +1,115 @@ +/* + * 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.column; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; + +public class Types +{ + private static final String ARRAY_PREFIX = "ARRAY<"; + private static final String COMPLEX_PREFIX = "COMPLEX<"; + + /** + * Create a {@link TypeSignature} given the value of {@link TypeSignature#asTypeString()} and a {@link TypeFactory} + */ + @Nullable + public static > T fromString(TypeFactory typeFactory, @Nullable String typeString) + { + if (typeString == null) { + return null; + } + switch (StringUtils.toUpperCase(typeString)) { + case "STRING": + return typeFactory.ofString(); + case "LONG": + return typeFactory.ofLong(); + case "FLOAT": + return typeFactory.ofFloat(); + case "DOUBLE": + return typeFactory.ofDouble(); + case "STRING_ARRAY": + return typeFactory.ofArray(typeFactory.ofString()); + case "LONG_ARRAY": + return typeFactory.ofArray(typeFactory.ofLong()); + case "DOUBLE_ARRAY": + return typeFactory.ofArray(typeFactory.ofDouble()); + case "COMPLEX": + return typeFactory.ofComplex(null); + default: + // we do not convert to uppercase here, because complex type name must be preserved in original casing + // array could be converted, but are not for no particular reason other than less spooky magic + if (typeString.startsWith(ARRAY_PREFIX)) { + T elementType = fromString(typeFactory, typeString.substring(ARRAY_PREFIX.length(), typeString.length() - 1)); + Preconditions.checkNotNull(elementType, "Array element type must not be null"); + return typeFactory.ofArray(elementType); + } + if (typeString.startsWith(COMPLEX_PREFIX)) { + return typeFactory.ofComplex(typeString.substring(COMPLEX_PREFIX.length(), typeString.length() - 1)); + } + } + return null; + } + + /** + * Returns true if {@link TypeSignature#getType()} is of the specified {@link TypeDescriptor} + */ + public static boolean is(@Nullable TypeSignature typeSignature, T typeDescriptor) + { + return typeSignature != null && typeSignature.is(typeDescriptor); + } + + /** + * Returns true if {@link TypeSignature#getType()} is null, or of the specified {@link TypeDescriptor} + */ + public static boolean isNullOr(@Nullable TypeSignature typeSignature, T typeDescriptor) + { + return typeSignature == null || typeSignature.is(typeDescriptor); + } + + /** + * Returns true if the {@link TypeSignature} is null, or is any one of the specified {@link TypeDescriptor} + */ + public static boolean isNullOrAnyOf( + @Nullable TypeSignature typeSignature, + T... typeDescriptors + ) + { + return typeSignature == null || typeSignature.anyOf(typeDescriptors); + } + + /** + * Returns true if either supplied {@link TypeSignature#getType()} is the given {@link TypeDescriptor} + * + * Useful for choosing a common {@link TypeDescriptor} between two {@link TypeSignature} when one of the signatures + * might be null. + */ + public static boolean either( + @Nullable TypeSignature typeSignature1, + @Nullable TypeSignature typeSignature2, + T typeDescriptor + ) + { + return (typeSignature1 != null && typeSignature1.is(typeDescriptor)) || + (typeSignature2 != null && typeSignature2.is(typeDescriptor)); + } +} diff --git a/core/src/main/java/org/apache/druid/segment/column/ValueType.java b/core/src/main/java/org/apache/druid/segment/column/ValueType.java index fa5e4894ef2f..86a82cd4470a 100644 --- a/core/src/main/java/org/apache/druid/segment/column/ValueType.java +++ b/core/src/main/java/org/apache/druid/segment/column/ValueType.java @@ -25,17 +25,21 @@ import javax.annotation.Nullable; /** - * This enumeration defines the Druid type system used to indicate the type of data stored in columns and produced by - * expressions and aggregations, used to allow query processing engine algorithms to compute results, used to compute query result - * row signatures, and all other type needs. + * This enumeration defines the high level classification of the Druid type system, used by {@link ColumnType} to + * indicate the type of data stored in columns and produced by expressions and aggregations, used to allow query + * processing engine algorithms to compute results, used to compute query result row signatures, and all other type + * needs. * - * Currently only the primitive types ({@link #isPrimitive()} is true) and {@link #COMPLEX} can be stored in columns + * Currently, only the primitive types ({@link #isPrimitive()} is true) and {@link #COMPLEX} can be stored in columns * and are also the only types handled directly by the query engines. Array types can currently be produced by * expressions and by some post-aggregators, but do not currently have special engine handling, and should be used by * implementors sparingly until full engine support is in place. Aggregators should never specify array types as their * output type until the engines fully support these types. + * + * @see ColumnType + * @see TypeSignature */ -public enum ValueType +public enum ValueType implements TypeDescriptor { /** * 64-bit double precision floating point number primitive type. This type may be used as a grouping key, or as an @@ -62,22 +66,6 @@ public enum ValueType * through this enum alone, and must be accompany this type indicator to properly handle. */ STRING, - /** - * Array object of 64-bit double precision floating point numbers. This type is not currently supported as a grouping - * key for aggregations, cannot be used as an input for numerical primitive aggregations such as sums, and may have - * limited support as an input among complex type sketch aggregators. - */ - DOUBLE_ARRAY, - /** - * Array object of 64-bit integer numbers. This type is not currently supported as a grouping key for aggregations, - * and may have limited support as an input among complex type sketch aggregators. - */ - LONG_ARRAY, - /** - * Array object of String objects. This type is not currently supported as a grouping key for aggregations, - * and may have limited support as an input among complex type sketch aggregators. - */ - STRING_ARRAY, /** * Placeholder for arbitrary 'complex' types, which have a corresponding serializer/deserializer implementation. Note * that knowing a type is complex alone isn't enough information to work with it directly, and additional information @@ -86,12 +74,20 @@ public enum ValueType * input to expression virtual columns, and might only be supported by the specific aggregators crafted to handle * this complex type. */ - COMPLEX; + COMPLEX, + + /** + * Placeholder for arbitrary arrays of other {@link ValueType}. This type is not currently supported as a grouping + * key for aggregations, cannot be used as an input for numerical primitive aggregations such as sums, and may have + * limited support as an input among complex type sketch aggregators. + */ + ARRAY; /** * Type is a numeric type, not including numeric array types */ + @Override public boolean isNumeric() { return isNumeric(this); @@ -100,6 +96,7 @@ public boolean isNumeric() /** * Type is an array type */ + @Override public boolean isArray() { return isArray(this); @@ -107,11 +104,12 @@ public boolean isArray() /** * Type is a 'primitive' type, which includes the {@link #isNumeric} types and {@link #STRING}, but not - * {@link #COMPLEX} or array types. + * {@link #COMPLEX} or {@link #ARRAY} types. * * Primitive types support being used for grouping to compute aggregates in both group by and top-n query engines, - * while non-primitive types currently do not + * while non-primitive types currently do not. */ + @Override public boolean isPrimitive() { return this.equals(ValueType.STRING) || isNumeric(this); @@ -134,6 +132,6 @@ public static boolean isNumeric(ValueType type) public static boolean isArray(ValueType type) { - return type == ValueType.DOUBLE_ARRAY || type == ValueType.LONG_ARRAY || type == ValueType.STRING_ARRAY; + return type == ValueType.ARRAY; } } diff --git a/core/src/test/java/org/apache/druid/math/expr/EvalTest.java b/core/src/test/java/org/apache/druid/math/expr/EvalTest.java index 82624b99d3b5..2f7068381265 100644 --- a/core/src/test/java/org/apache/druid/math/expr/EvalTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/EvalTest.java @@ -38,14 +38,14 @@ public class EvalTest extends InitializedNullHandlingTest private long evalLong(String x, Expr.ObjectBinding bindings) { ExprEval ret = eval(x, bindings); - Assert.assertEquals(ExprType.LONG, ret.type()); + Assert.assertEquals(ExpressionType.LONG, ret.type()); return ret.asLong(); } private double evalDouble(String x, Expr.ObjectBinding bindings) { ExprEval ret = eval(x, bindings); - Assert.assertEquals(ExprType.DOUBLE, ret.type()); + Assert.assertEquals(ExpressionType.DOUBLE, ret.type()); return ret.asDouble(); } @@ -198,22 +198,22 @@ public void testArrayToScalar() Assert.assertEquals(false, ExprEval.ofStringArray(new String[]{"1", "2.2"}).asBoolean()); // test casting arrays to scalars - Assert.assertEquals(1L, ExprEval.ofLongArray(new Long[]{1L}).castTo(ExprType.LONG).value()); - Assert.assertEquals(NullHandling.defaultLongValue(), ExprEval.ofLongArray(new Long[]{null}).castTo(ExprType.LONG).value()); - Assert.assertEquals(1.0, ExprEval.ofLongArray(new Long[]{1L}).castTo(ExprType.DOUBLE).asDouble(), 0.0); - Assert.assertEquals("1", ExprEval.ofLongArray(new Long[]{1L}).castTo(ExprType.STRING).value()); - - Assert.assertEquals(1.1, ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExprType.DOUBLE).asDouble(), 0.0); - Assert.assertEquals(NullHandling.defaultDoubleValue(), ExprEval.ofDoubleArray(new Double[]{null}).castTo(ExprType.DOUBLE).value()); - Assert.assertEquals(1L, ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExprType.LONG).value()); - Assert.assertEquals("1.1", ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExprType.STRING).value()); - - Assert.assertEquals("foo", ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExprType.STRING).value()); - Assert.assertEquals(NullHandling.defaultLongValue(), ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExprType.LONG).value()); - Assert.assertEquals(NullHandling.defaultDoubleValue(), ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExprType.DOUBLE).value()); - Assert.assertEquals("1", ExprEval.ofStringArray(new String[]{"1"}).castTo(ExprType.STRING).value()); - Assert.assertEquals(1L, ExprEval.ofStringArray(new String[]{"1"}).castTo(ExprType.LONG).value()); - Assert.assertEquals(1.0, ExprEval.ofStringArray(new String[]{"1"}).castTo(ExprType.DOUBLE).value()); + Assert.assertEquals(1L, ExprEval.ofLongArray(new Long[]{1L}).castTo(ExpressionType.LONG).value()); + Assert.assertEquals(NullHandling.defaultLongValue(), ExprEval.ofLongArray(new Long[]{null}).castTo(ExpressionType.LONG).value()); + Assert.assertEquals(1.0, ExprEval.ofLongArray(new Long[]{1L}).castTo(ExpressionType.DOUBLE).asDouble(), 0.0); + Assert.assertEquals("1", ExprEval.ofLongArray(new Long[]{1L}).castTo(ExpressionType.STRING).value()); + + Assert.assertEquals(1.1, ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExpressionType.DOUBLE).asDouble(), 0.0); + Assert.assertEquals(NullHandling.defaultDoubleValue(), ExprEval.ofDoubleArray(new Double[]{null}).castTo(ExpressionType.DOUBLE).value()); + Assert.assertEquals(1L, ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExpressionType.LONG).value()); + Assert.assertEquals("1.1", ExprEval.ofDoubleArray(new Double[]{1.1}).castTo(ExpressionType.STRING).value()); + + Assert.assertEquals("foo", ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExpressionType.STRING).value()); + Assert.assertEquals(NullHandling.defaultLongValue(), ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExpressionType.LONG).value()); + Assert.assertEquals(NullHandling.defaultDoubleValue(), ExprEval.ofStringArray(new String[]{"foo"}).castTo(ExpressionType.DOUBLE).value()); + Assert.assertEquals("1", ExprEval.ofStringArray(new String[]{"1"}).castTo(ExpressionType.STRING).value()); + Assert.assertEquals(1L, ExprEval.ofStringArray(new String[]{"1"}).castTo(ExpressionType.LONG).value()); + Assert.assertEquals(1.0, ExprEval.ofStringArray(new String[]{"1"}).castTo(ExpressionType.DOUBLE).value()); } @Test @@ -221,7 +221,7 @@ public void testStringArrayToScalarStringBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type STRING"); - ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExprType.STRING); + ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.STRING); } @Test @@ -229,7 +229,7 @@ public void testStringArrayToScalarLongBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type LONG"); - ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExprType.LONG); + ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.LONG); } @Test @@ -237,7 +237,7 @@ public void testStringArrayToScalarDoubleBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type DOUBLE"); - ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExprType.DOUBLE); + ExprEval.ofStringArray(new String[]{"foo", "bar"}).castTo(ExpressionType.DOUBLE); } @Test @@ -245,7 +245,7 @@ public void testLongArrayToScalarStringBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type STRING"); - ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExprType.STRING); + ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.STRING); } @Test @@ -253,7 +253,7 @@ public void testLongArrayToScalarLongBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type LONG"); - ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExprType.LONG); + ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.LONG); } @Test @@ -261,7 +261,7 @@ public void testLongArrayToScalarDoubleBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type DOUBLE"); - ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExprType.DOUBLE); + ExprEval.ofLongArray(new Long[]{1L, 2L}).castTo(ExpressionType.DOUBLE); } @Test @@ -269,7 +269,7 @@ public void testDoubleArrayToScalarStringBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type STRING"); - ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExprType.STRING); + ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.STRING); } @Test @@ -277,7 +277,7 @@ public void testDoubleArrayToScalarLongBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type LONG"); - ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExprType.LONG); + ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.LONG); } @Test @@ -285,7 +285,7 @@ public void testDoubleArrayToScalarDoubleBadCast() { expectedException.expect(IAE.class); expectedException.expectMessage("invalid type DOUBLE"); - ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExprType.DOUBLE); + ExprEval.ofDoubleArray(new Double[]{1.1, 2.2}).castTo(ExpressionType.DOUBLE); } @Test @@ -350,26 +350,26 @@ public void testBooleanReturn() ); ExprEval eval = Parser.parse("x==y", ExprMacroTable.nil()).eval(bindings); Assert.assertTrue(eval.asBoolean()); - Assert.assertEquals(ExprType.LONG, eval.type()); + Assert.assertEquals(ExpressionType.LONG, eval.type()); eval = Parser.parse("x!=y", ExprMacroTable.nil()).eval(bindings); Assert.assertFalse(eval.asBoolean()); - Assert.assertEquals(ExprType.LONG, eval.type()); + Assert.assertEquals(ExpressionType.LONG, eval.type()); eval = Parser.parse("x==z", ExprMacroTable.nil()).eval(bindings); Assert.assertTrue(eval.asBoolean()); - Assert.assertEquals(ExprType.DOUBLE, eval.type()); + Assert.assertEquals(ExpressionType.DOUBLE, eval.type()); eval = Parser.parse("x!=z", ExprMacroTable.nil()).eval(bindings); Assert.assertFalse(eval.asBoolean()); - Assert.assertEquals(ExprType.DOUBLE, eval.type()); + Assert.assertEquals(ExpressionType.DOUBLE, eval.type()); eval = Parser.parse("z==w", ExprMacroTable.nil()).eval(bindings); Assert.assertTrue(eval.asBoolean()); - Assert.assertEquals(ExprType.DOUBLE, eval.type()); + Assert.assertEquals(ExpressionType.DOUBLE, eval.type()); eval = Parser.parse("z!=w", ExprMacroTable.nil()).eval(bindings); Assert.assertFalse(eval.asBoolean()); - Assert.assertEquals(ExprType.DOUBLE, eval.type()); + Assert.assertEquals(ExpressionType.DOUBLE, eval.type()); } } diff --git a/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java b/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java index fa3699d22afc..a486586c9073 100644 --- a/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java @@ -56,7 +56,7 @@ public void testStringSerdeTooBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.STRING, + ExpressionType.STRING, 16, 10 )); @@ -94,8 +94,8 @@ public void testStringArraySerdeToBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.STRING_ARRAY, - 14, + ExpressionType.STRING_ARRAY, + 15, 10 )); assertExpr(0, ExprEval.ofStringArray(new String[]{"hello", "hi", "hey"}), 10); @@ -108,8 +108,8 @@ public void testStringArrayEvalToBig() // this has a different failure size than string serde because it doesn't check incrementally expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.STRING_ARRAY, - 27, + ExpressionType.STRING_ARRAY, + 28, 10 )); assertEstimatedBytes(ExprEval.ofStringArray(new String[]{"hello", "hi", "hey"}), 10); @@ -129,8 +129,8 @@ public void testLongArraySerdeTooBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.LONG_ARRAY, - 29, + ExpressionType.LONG_ARRAY, + 30, 10 )); assertExpr(0, ExprEval.ofLongArray(new Long[]{1L, 2L, 3L}), 10); @@ -142,8 +142,8 @@ public void testLongArrayEvalTooBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.LONG_ARRAY, - NullHandling.sqlCompatible() ? 32 : 29, + ExpressionType.LONG_ARRAY, + NullHandling.sqlCompatible() ? 33 : 30, 10 )); assertEstimatedBytes(ExprEval.ofLongArray(new Long[]{1L, 2L, 3L}), 10); @@ -163,8 +163,8 @@ public void testDoubleArraySerdeTooBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.DOUBLE_ARRAY, - 29, + ExpressionType.DOUBLE_ARRAY, + 30, 10 )); assertExpr(0, ExprEval.ofDoubleArray(new Double[]{1.1, 2.2, 3.3}), 10); @@ -176,8 +176,8 @@ public void testDoubleArrayEvalTooBig() expectedException.expect(ISE.class); expectedException.expectMessage(StringUtils.format( "Unable to serialize [%s], size [%s] is larger than max [%s]", - ExprType.DOUBLE_ARRAY, - NullHandling.sqlCompatible() ? 32 : 29, + ExpressionType.DOUBLE_ARRAY, + NullHandling.sqlCompatible() ? 33 : 30, 10 )); assertEstimatedBytes(ExprEval.ofDoubleArray(new Double[]{1.1, 2.2, 3.3}), 10); @@ -296,7 +296,7 @@ private void assertExpr(int position, ExprEval expected) private void assertExpr(int position, ExprEval expected, int maxSizeBytes) { ExprEval.serialize(buffer, position, expected, maxSizeBytes); - if (ExprType.isArray(expected.type())) { + if (expected.type().isArray()) { Assert.assertArrayEquals( expected.asArray(), ExprEval.deserialize(buffer, position + 1, ExprType.fromByte(buffer.get(position))).asArray() diff --git a/core/src/test/java/org/apache/druid/math/expr/ExprTest.java b/core/src/test/java/org/apache/druid/math/expr/ExprTest.java index 25815591190b..05c192923a75 100644 --- a/core/src/test/java/org/apache/druid/math/expr/ExprTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/ExprTest.java @@ -134,6 +134,7 @@ public void testEqualsContractForStringExpr() { EqualsVerifier.forClass(StringExpr.class) .withIgnoredFields("outputType") + .withPrefabValues(ExpressionType.class, ExpressionType.STRING, ExpressionType.DOUBLE) .usingGetClass() .verify(); } @@ -143,6 +144,7 @@ public void testEqualsContractForDoubleExpr() { EqualsVerifier.forClass(DoubleExpr.class) .withIgnoredFields("outputType") + .withPrefabValues(ExpressionType.class, ExpressionType.DOUBLE, ExpressionType.LONG) .usingGetClass() .verify(); } @@ -152,6 +154,7 @@ public void testEqualsContractForLongExpr() { EqualsVerifier.forClass(LongExpr.class) .withIgnoredFields("outputType") + .withPrefabValues(ExpressionType.class, ExpressionType.LONG, ExpressionType.STRING) .usingGetClass() .verify(); } @@ -162,6 +165,7 @@ public void testEqualsContractForStringArrayExpr() EqualsVerifier.forClass(StringArrayExpr.class) .withIgnoredFields("outputType") .withPrefabValues(Object.class, new String[]{"foo"}, new String[0]) + .withPrefabValues(ExpressionType.class, ExpressionType.STRING_ARRAY, ExpressionType.LONG_ARRAY) .usingGetClass() .verify(); } @@ -172,6 +176,7 @@ public void testEqualsContractForLongArrayExpr() EqualsVerifier.forClass(LongArrayExpr.class) .withIgnoredFields("outputType") .withPrefabValues(Object.class, new Long[]{1L}, new Long[0]) + .withPrefabValues(ExpressionType.class, ExpressionType.LONG_ARRAY, ExpressionType.DOUBLE_ARRAY) .usingGetClass() .verify(); } @@ -182,6 +187,7 @@ public void testEqualsContractForDoubleArrayExpr() EqualsVerifier.forClass(DoubleArrayExpr.class) .withIgnoredFields("outputType") .withPrefabValues(Object.class, new Double[]{1.0}, new Double[0]) + .withPrefabValues(ExpressionType.class, ExpressionType.DOUBLE_ARRAY, ExpressionType.STRING_ARRAY) .usingGetClass() .verify(); } @@ -202,6 +208,7 @@ public void testEqualsContractForNullLongExpr() { EqualsVerifier.forClass(NullLongExpr.class) .withIgnoredFields("outputType", "value") + .withPrefabValues(ExpressionType.class, ExpressionType.LONG, ExpressionType.STRING) .verify(); } @@ -210,6 +217,7 @@ public void testEqualsContractForNullDoubleExpr() { EqualsVerifier.forClass(NullDoubleExpr.class) .withIgnoredFields("outputType", "value") + .withPrefabValues(ExpressionType.class, ExpressionType.DOUBLE, ExpressionType.STRING) .verify(); } } diff --git a/core/src/test/java/org/apache/druid/math/expr/ExpressionTypeTest.java b/core/src/test/java/org/apache/druid/math/expr/ExpressionTypeTest.java new file mode 100644 index 000000000000..2f6a4bd4dca2 --- /dev/null +++ b/core/src/test/java/org/apache/druid/math/expr/ExpressionTypeTest.java @@ -0,0 +1,197 @@ +/* + * 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.math.expr; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class ExpressionTypeTest +{ + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final ExpressionType SOME_COMPLEX = new ExpressionType(ExprType.COMPLEX, "foo", null); + + @Rule + public final ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testSerde() throws JsonProcessingException + { + Assert.assertEquals(ExpressionType.STRING, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.STRING), ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.LONG), ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.DOUBLE), ExpressionType.class)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.STRING_ARRAY), ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.LONG_ARRAY), ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ExpressionType.DOUBLE_ARRAY), ExpressionType.class)); + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue(MAPPER.writeValueAsString(SOME_COMPLEX), ExpressionType.class)); + } + + @Test + public void testSerdeFromString() throws JsonProcessingException + { + Assert.assertEquals(ExpressionType.STRING, MAPPER.readValue("\"STRING\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG, MAPPER.readValue("\"LONG\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE, MAPPER.readValue("\"DOUBLE\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue("\"ARRAY\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue("\"ARRAY\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue("\"ARRAY\"", ExpressionType.class)); + + ExpressionType whatHaveIdone = new ExpressionType(ExprType.ARRAY, null, new ExpressionType(ExprType.ARRAY, null, SOME_COMPLEX)); + Assert.assertEquals(whatHaveIdone, MAPPER.readValue("\"ARRAY>>\"", ExpressionType.class)); + + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue("\"COMPLEX\"", ExpressionType.class)); + // make sure legacy works too + Assert.assertEquals(ExpressionType.STRING, MAPPER.readValue("\"string\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG, MAPPER.readValue("\"long\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE, MAPPER.readValue("\"double\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue("\"STRING_ARRAY\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue("\"LONG_ARRAY\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue("\"DOUBLE_ARRAY\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue("\"string_array\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue("\"long_array\"", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue("\"double_array\"", ExpressionType.class)); + // ARRAY<*> and COMPLEX<*> patterns must match exactly ... + Assert.assertNotEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue("\"array\"", ExpressionType.class)); + Assert.assertNotEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue("\"array\"", ExpressionType.class)); + Assert.assertNotEquals(SOME_COMPLEX, MAPPER.readValue("\"COMPLEX\"", ExpressionType.class)); + // this works though because array recursively calls on element type... + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue("\"ARRAY\"", ExpressionType.class)); + } + + @Test + public void testFutureProof() throws JsonProcessingException + { + // in case we ever want to switch from string serde to JSON objects for type info, be ready + Assert.assertEquals(ExpressionType.STRING, MAPPER.readValue("{\"type\":\"STRING\"}", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG, MAPPER.readValue("{\"type\":\"LONG\"}", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE, MAPPER.readValue("{\"type\":\"DOUBLE\"}", ExpressionType.class)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"STRING\"}}", ExpressionType.class)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"LONG\"}}", ExpressionType.class)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"DOUBLE\"}}", ExpressionType.class)); + + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue("{\"type\":\"COMPLEX\", \"complexTypeName\":\"foo\"}", ExpressionType.class)); + + ExpressionType whatHaveIdone = new ExpressionType(ExprType.ARRAY, null, new ExpressionType(ExprType.ARRAY, null, SOME_COMPLEX)); + Assert.assertEquals(whatHaveIdone, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"COMPLEX\", \"complexTypeName\":\"foo\"}}}", ExpressionType.class)); + } + + @Test + public void testConvertFromColumnType() + { + Assert.assertNull(ExpressionType.fromColumnType(null)); + Assert.assertEquals(ExpressionType.LONG, ExpressionType.fromColumnType(ColumnType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionType.fromColumnType(ColumnType.FLOAT)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionType.fromColumnType(ColumnType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionType.fromColumnType(ColumnType.STRING)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, ExpressionType.fromColumnType(ColumnType.LONG_ARRAY)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, ExpressionType.fromColumnType(ColumnType.DOUBLE_ARRAY)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, ExpressionType.fromColumnType(ColumnType.STRING_ARRAY)); + Assert.assertEquals( + SOME_COMPLEX, + ExpressionType.fromColumnType(ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName())) + ); + ExpressionType complexArray = new ExpressionType( + ExprType.ARRAY, + null, + new ExpressionType(ExprType.ARRAY, null, SOME_COMPLEX) + ); + ColumnType complexArrayColumn = new ColumnType( + ValueType.ARRAY, + null, + new ColumnType(ValueType.ARRAY, + null, + ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName()) + ) + ); + Assert.assertEquals(complexArray, ExpressionType.fromColumnType(complexArrayColumn)); + } + + @Test + public void testConvertFromColumnTypeStrict() + { + Assert.assertEquals(ExpressionType.LONG, ExpressionType.fromColumnTypeStrict(ColumnType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionType.fromColumnTypeStrict(ColumnType.FLOAT)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionType.fromColumnTypeStrict(ColumnType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionType.fromColumnTypeStrict(ColumnType.STRING)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, ExpressionType.fromColumnTypeStrict(ColumnType.LONG_ARRAY)); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, ExpressionType.fromColumnTypeStrict(ColumnType.DOUBLE_ARRAY)); + Assert.assertEquals(ExpressionType.STRING_ARRAY, ExpressionType.fromColumnTypeStrict(ColumnType.STRING_ARRAY)); + Assert.assertEquals( + SOME_COMPLEX, + ExpressionType.fromColumnTypeStrict(ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName())) + ); + ExpressionType complexArray = new ExpressionType( + ExprType.ARRAY, + null, + new ExpressionType(ExprType.ARRAY, null, SOME_COMPLEX) + ); + ColumnType complexArrayColumn = new ColumnType( + ValueType.ARRAY, + null, + new ColumnType(ValueType.ARRAY, + null, + ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName()) + ) + ); + Assert.assertEquals(complexArray, ExpressionType.fromColumnTypeStrict(complexArrayColumn)); + } + + @Test + public void testConvertFromColumnTypeStrictNull() + { + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Unsupported unknown value type"); + ExpressionType.fromColumnTypeStrict(null); + } + + @Test + public void testConvertToColumnType() + { + Assert.assertEquals(ColumnType.LONG, ExpressionType.toColumnType(ExpressionType.LONG)); + Assert.assertEquals(ColumnType.DOUBLE, ExpressionType.toColumnType(ExpressionType.DOUBLE)); + Assert.assertEquals(ColumnType.STRING, ExpressionType.toColumnType(ExpressionType.STRING)); + Assert.assertEquals(ColumnType.LONG_ARRAY, ExpressionType.toColumnType(ExpressionType.LONG_ARRAY)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, ExpressionType.toColumnType(ExpressionType.DOUBLE_ARRAY)); + Assert.assertEquals(ColumnType.STRING_ARRAY, ExpressionType.toColumnType(ExpressionType.STRING_ARRAY)); + Assert.assertEquals( + ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName()), + ExpressionType.toColumnType(SOME_COMPLEX) + ); + ExpressionType complexArray = new ExpressionType( + ExprType.ARRAY, + null, + new ExpressionType(ExprType.ARRAY, null, SOME_COMPLEX) + ); + ColumnType complexArrayColumn = new ColumnType( + ValueType.ARRAY, + null, + new ColumnType(ValueType.ARRAY, + null, + ColumnType.ofComplex(SOME_COMPLEX.getComplexTypeName()) + ) + ); + Assert.assertEquals(complexArrayColumn, ExpressionType.toColumnType(complexArray)); + } +} diff --git a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java index 82d7fbaed825..c05422a840bd 100644 --- a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -441,9 +441,9 @@ public void testRoundWithNullValueOrInvalid() Set> invalidArguments = ImmutableSet.of( Pair.of("null", "STRING"), Pair.of("x", "STRING"), - Pair.of("b", "LONG_ARRAY"), - Pair.of("c", "DOUBLE_ARRAY"), - Pair.of("a", "STRING_ARRAY") + Pair.of("b", "ARRAY"), + Pair.of("c", "ARRAY"), + Pair.of("a", "ARRAY") ); for (Pair argAndType : invalidArguments) { if (NullHandling.sqlCompatible()) { @@ -472,8 +472,8 @@ public void testRoundWithInvalidSecondArgument() Set> invalidArguments = ImmutableSet.of( Pair.of("1.2", "DOUBLE"), Pair.of("x", "STRING"), - Pair.of("a", "STRING_ARRAY"), - Pair.of("c", "DOUBLE_ARRAY") + Pair.of("a", "ARRAY"), + Pair.of("c", "ARRAY") ); for (Pair argAndType : invalidArguments) { @@ -512,7 +512,7 @@ public void testGreatest() Assert.fail("Did not throw IllegalArgumentException"); } catch (IllegalArgumentException e) { - Assert.assertEquals("Function[greatest] does not accept STRING_ARRAY types", e.getMessage()); + Assert.assertEquals("Function[greatest] does not accept ARRAY types", e.getMessage()); } // Null handling @@ -540,7 +540,7 @@ public void testLeast() Assert.fail("Did not throw IllegalArgumentException"); } catch (IllegalArgumentException e) { - Assert.assertEquals("Function[least] does not accept LONG_ARRAY types", e.getMessage()); + Assert.assertEquals("Function[least] does not accept ARRAY types", e.getMessage()); } // Null handling diff --git a/core/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java b/core/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java index 0c8ea1b71fab..07d523a0c816 100644 --- a/core/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java @@ -32,18 +32,18 @@ public class OutputTypeTest extends InitializedNullHandlingTest { private final Expr.InputBindingInspector inspector = inspectorFromMap( - ImmutableMap.builder().put("x", ExprType.STRING) - .put("x_", ExprType.STRING) - .put("y", ExprType.LONG) - .put("y_", ExprType.LONG) - .put("z", ExprType.DOUBLE) - .put("z_", ExprType.DOUBLE) - .put("a", ExprType.STRING_ARRAY) - .put("a_", ExprType.STRING_ARRAY) - .put("b", ExprType.LONG_ARRAY) - .put("b_", ExprType.LONG_ARRAY) - .put("c", ExprType.DOUBLE_ARRAY) - .put("c_", ExprType.DOUBLE_ARRAY) + ImmutableMap.builder().put("x", ExpressionType.STRING) + .put("x_", ExpressionType.STRING) + .put("y", ExpressionType.LONG) + .put("y_", ExpressionType.LONG) + .put("z", ExpressionType.DOUBLE) + .put("z_", ExpressionType.DOUBLE) + .put("a", ExpressionType.STRING_ARRAY) + .put("a_", ExpressionType.STRING_ARRAY) + .put("b", ExpressionType.LONG_ARRAY) + .put("b_", ExpressionType.LONG_ARRAY) + .put("c", ExpressionType.DOUBLE_ARRAY) + .put("c_", ExpressionType.DOUBLE_ARRAY) .build() ); @@ -53,339 +53,339 @@ public class OutputTypeTest extends InitializedNullHandlingTest @Test public void testConstantsAndIdentifiers() { - assertOutputType("'hello'", inspector, ExprType.STRING); - assertOutputType("23", inspector, ExprType.LONG); - assertOutputType("3.2", inspector, ExprType.DOUBLE); - assertOutputType("['a', 'b']", inspector, ExprType.STRING_ARRAY); - assertOutputType("[1,2,3]", inspector, ExprType.LONG_ARRAY); - assertOutputType("[1.0]", inspector, ExprType.DOUBLE_ARRAY); - assertOutputType("x", inspector, ExprType.STRING); - assertOutputType("y", inspector, ExprType.LONG); - assertOutputType("z", inspector, ExprType.DOUBLE); - assertOutputType("a", inspector, ExprType.STRING_ARRAY); - assertOutputType("b", inspector, ExprType.LONG_ARRAY); - assertOutputType("c", inspector, ExprType.DOUBLE_ARRAY); + assertOutputType("'hello'", inspector, ExpressionType.STRING); + assertOutputType("23", inspector, ExpressionType.LONG); + assertOutputType("3.2", inspector, ExpressionType.DOUBLE); + assertOutputType("['a', 'b']", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("[1,2,3]", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("[1.0]", inspector, ExpressionType.DOUBLE_ARRAY); + assertOutputType("x", inspector, ExpressionType.STRING); + assertOutputType("y", inspector, ExpressionType.LONG); + assertOutputType("z", inspector, ExpressionType.DOUBLE); + assertOutputType("a", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("b", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("c", inspector, ExpressionType.DOUBLE_ARRAY); } @Test public void testUnaryOperators() { - assertOutputType("-1", inspector, ExprType.LONG); - assertOutputType("-1.1", inspector, ExprType.DOUBLE); - assertOutputType("-y", inspector, ExprType.LONG); - assertOutputType("-z", inspector, ExprType.DOUBLE); - - assertOutputType("!'true'", inspector, ExprType.LONG); - assertOutputType("!1", inspector, ExprType.LONG); - assertOutputType("!1.1", inspector, ExprType.DOUBLE); - assertOutputType("!x", inspector, ExprType.LONG); - assertOutputType("!y", inspector, ExprType.LONG); - assertOutputType("!z", inspector, ExprType.DOUBLE); + assertOutputType("-1", inspector, ExpressionType.LONG); + assertOutputType("-1.1", inspector, ExpressionType.DOUBLE); + assertOutputType("-y", inspector, ExpressionType.LONG); + assertOutputType("-z", inspector, ExpressionType.DOUBLE); + + assertOutputType("!'true'", inspector, ExpressionType.LONG); + assertOutputType("!1", inspector, ExpressionType.LONG); + assertOutputType("!1.1", inspector, ExpressionType.DOUBLE); + assertOutputType("!x", inspector, ExpressionType.LONG); + assertOutputType("!y", inspector, ExpressionType.LONG); + assertOutputType("!z", inspector, ExpressionType.DOUBLE); } @Test public void testBinaryMathOperators() { - assertOutputType("1+1", inspector, ExprType.LONG); - assertOutputType("1-1", inspector, ExprType.LONG); - assertOutputType("1*1", inspector, ExprType.LONG); - assertOutputType("1/1", inspector, ExprType.LONG); - assertOutputType("1^1", inspector, ExprType.LONG); - assertOutputType("1%1", inspector, ExprType.LONG); - - assertOutputType("y+y_", inspector, ExprType.LONG); - assertOutputType("y-y_", inspector, ExprType.LONG); - assertOutputType("y*y_", inspector, ExprType.LONG); - assertOutputType("y/y_", inspector, ExprType.LONG); - assertOutputType("y^y_", inspector, ExprType.LONG); - assertOutputType("y%y_", inspector, ExprType.LONG); - - assertOutputType("y+z", inspector, ExprType.DOUBLE); - assertOutputType("y-z", inspector, ExprType.DOUBLE); - assertOutputType("y*z", inspector, ExprType.DOUBLE); - assertOutputType("y/z", inspector, ExprType.DOUBLE); - assertOutputType("y^z", inspector, ExprType.DOUBLE); - assertOutputType("y%z", inspector, ExprType.DOUBLE); - - assertOutputType("z+z_", inspector, ExprType.DOUBLE); - assertOutputType("z-z_", inspector, ExprType.DOUBLE); - assertOutputType("z*z_", inspector, ExprType.DOUBLE); - assertOutputType("z/z_", inspector, ExprType.DOUBLE); - assertOutputType("z^z_", inspector, ExprType.DOUBLE); - assertOutputType("z%z_", inspector, ExprType.DOUBLE); - - assertOutputType("y>y_", inspector, ExprType.LONG); - assertOutputType("y_=y", inspector, ExprType.LONG); - assertOutputType("y_==y", inspector, ExprType.LONG); - assertOutputType("y_!=y", inspector, ExprType.LONG); - assertOutputType("y_ && y", inspector, ExprType.LONG); - assertOutputType("y_ || y", inspector, ExprType.LONG); - - assertOutputType("z>y_", inspector, ExprType.DOUBLE); - assertOutputType("z=z", inspector, ExprType.DOUBLE); - assertOutputType("z==y", inspector, ExprType.DOUBLE); - assertOutputType("z!=y", inspector, ExprType.DOUBLE); - assertOutputType("z && y", inspector, ExprType.DOUBLE); - assertOutputType("y || z", inspector, ExprType.DOUBLE); - - assertOutputType("z>z_", inspector, ExprType.DOUBLE); - assertOutputType("z=z", inspector, ExprType.DOUBLE); - assertOutputType("z==z_", inspector, ExprType.DOUBLE); - assertOutputType("z!=z_", inspector, ExprType.DOUBLE); - assertOutputType("z && z_", inspector, ExprType.DOUBLE); - assertOutputType("z_ || z", inspector, ExprType.DOUBLE); - - assertOutputType("1*(2 + 3.0)", inspector, ExprType.DOUBLE); + assertOutputType("1+1", inspector, ExpressionType.LONG); + assertOutputType("1-1", inspector, ExpressionType.LONG); + assertOutputType("1*1", inspector, ExpressionType.LONG); + assertOutputType("1/1", inspector, ExpressionType.LONG); + assertOutputType("1^1", inspector, ExpressionType.LONG); + assertOutputType("1%1", inspector, ExpressionType.LONG); + + assertOutputType("y+y_", inspector, ExpressionType.LONG); + assertOutputType("y-y_", inspector, ExpressionType.LONG); + assertOutputType("y*y_", inspector, ExpressionType.LONG); + assertOutputType("y/y_", inspector, ExpressionType.LONG); + assertOutputType("y^y_", inspector, ExpressionType.LONG); + assertOutputType("y%y_", inspector, ExpressionType.LONG); + + assertOutputType("y+z", inspector, ExpressionType.DOUBLE); + assertOutputType("y-z", inspector, ExpressionType.DOUBLE); + assertOutputType("y*z", inspector, ExpressionType.DOUBLE); + assertOutputType("y/z", inspector, ExpressionType.DOUBLE); + assertOutputType("y^z", inspector, ExpressionType.DOUBLE); + assertOutputType("y%z", inspector, ExpressionType.DOUBLE); + + assertOutputType("z+z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z-z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z*z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z/z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z^z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z%z_", inspector, ExpressionType.DOUBLE); + + assertOutputType("y>y_", inspector, ExpressionType.LONG); + assertOutputType("y_=y", inspector, ExpressionType.LONG); + assertOutputType("y_==y", inspector, ExpressionType.LONG); + assertOutputType("y_!=y", inspector, ExpressionType.LONG); + assertOutputType("y_ && y", inspector, ExpressionType.LONG); + assertOutputType("y_ || y", inspector, ExpressionType.LONG); + + assertOutputType("z>y_", inspector, ExpressionType.DOUBLE); + assertOutputType("z=z", inspector, ExpressionType.DOUBLE); + assertOutputType("z==y", inspector, ExpressionType.DOUBLE); + assertOutputType("z!=y", inspector, ExpressionType.DOUBLE); + assertOutputType("z && y", inspector, ExpressionType.DOUBLE); + assertOutputType("y || z", inspector, ExpressionType.DOUBLE); + + assertOutputType("z>z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z=z", inspector, ExpressionType.DOUBLE); + assertOutputType("z==z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z!=z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z && z_", inspector, ExpressionType.DOUBLE); + assertOutputType("z_ || z", inspector, ExpressionType.DOUBLE); + + assertOutputType("1*(2 + 3.0)", inspector, ExpressionType.DOUBLE); } @Test public void testUnivariateMathFunctions() { - assertOutputType("pi()", inspector, ExprType.DOUBLE); - assertOutputType("abs(x)", inspector, ExprType.STRING); - assertOutputType("abs(y)", inspector, ExprType.LONG); - assertOutputType("abs(z)", inspector, ExprType.DOUBLE); - assertOutputType("cos(y)", inspector, ExprType.DOUBLE); - assertOutputType("cos(z)", inspector, ExprType.DOUBLE); + assertOutputType("pi()", inspector, ExpressionType.DOUBLE); + assertOutputType("abs(x)", inspector, ExpressionType.STRING); + assertOutputType("abs(y)", inspector, ExpressionType.LONG); + assertOutputType("abs(z)", inspector, ExpressionType.DOUBLE); + assertOutputType("cos(y)", inspector, ExpressionType.DOUBLE); + assertOutputType("cos(z)", inspector, ExpressionType.DOUBLE); } @Test public void testBivariateMathFunctions() { - assertOutputType("div(y,y_)", inspector, ExprType.LONG); - assertOutputType("div(y,z_)", inspector, ExprType.LONG); - assertOutputType("div(z,z_)", inspector, ExprType.LONG); + assertOutputType("div(y,y_)", inspector, ExpressionType.LONG); + assertOutputType("div(y,z_)", inspector, ExpressionType.LONG); + assertOutputType("div(z,z_)", inspector, ExpressionType.LONG); - assertOutputType("max(y,y_)", inspector, ExprType.LONG); - assertOutputType("max(y,z_)", inspector, ExprType.DOUBLE); - assertOutputType("max(z,z_)", inspector, ExprType.DOUBLE); + assertOutputType("max(y,y_)", inspector, ExpressionType.LONG); + assertOutputType("max(y,z_)", inspector, ExpressionType.DOUBLE); + assertOutputType("max(z,z_)", inspector, ExpressionType.DOUBLE); - assertOutputType("hypot(y,y_)", inspector, ExprType.DOUBLE); - assertOutputType("hypot(y,z_)", inspector, ExprType.DOUBLE); - assertOutputType("hypot(z,z_)", inspector, ExprType.DOUBLE); + assertOutputType("hypot(y,y_)", inspector, ExpressionType.DOUBLE); + assertOutputType("hypot(y,z_)", inspector, ExpressionType.DOUBLE); + assertOutputType("hypot(z,z_)", inspector, ExpressionType.DOUBLE); } @Test public void testConditionalFunctions() { - assertOutputType("if(y, 'foo', 'bar')", inspector, ExprType.STRING); - assertOutputType("if(y,2,3)", inspector, ExprType.LONG); - assertOutputType("if(y,2,3.0)", inspector, ExprType.DOUBLE); + assertOutputType("if(y, 'foo', 'bar')", inspector, ExpressionType.STRING); + assertOutputType("if(y,2,3)", inspector, ExpressionType.LONG); + assertOutputType("if(y,2,3.0)", inspector, ExpressionType.DOUBLE); assertOutputType( "case_simple(x,'baz','is baz','foo','is foo','is other')", inspector, - ExprType.STRING + ExpressionType.STRING ); assertOutputType( "case_simple(y,2,2,3,3,4)", inspector, - ExprType.LONG + ExpressionType.LONG ); assertOutputType( "case_simple(z,2.0,2.0,3.0,3.0,4.0)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); assertOutputType( "case_simple(y,2,2,3,3.0,4)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); assertOutputType( "case_simple(z,2.0,2.0,3.0,3.0,null)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); assertOutputType( "case_searched(x=='baz','is baz',x=='foo','is foo','is other')", inspector, - ExprType.STRING + ExpressionType.STRING ); assertOutputType( "case_searched(y==1,1,y==2,2,0)", inspector, - ExprType.LONG + ExpressionType.LONG ); assertOutputType( "case_searched(z==1.0,1.0,z==2.0,2.0,0.0)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); assertOutputType( "case_searched(y==1,1,y==2,2.0,0)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); assertOutputType( "case_searched(z==1.0,1,z==2.0,2,null)", inspector, - ExprType.LONG + ExpressionType.LONG ); assertOutputType( "case_searched(z==1.0,1.0,z==2.0,2.0,null)", inspector, - ExprType.DOUBLE + ExpressionType.DOUBLE ); - assertOutputType("nvl(x, 'foo')", inspector, ExprType.STRING); - assertOutputType("nvl(y, 1)", inspector, ExprType.LONG); - assertOutputType("nvl(y, 1.1)", inspector, ExprType.DOUBLE); - assertOutputType("nvl(z, 2.0)", inspector, ExprType.DOUBLE); - assertOutputType("nvl(y, 2.0)", inspector, ExprType.DOUBLE); - assertOutputType("isnull(x)", inspector, ExprType.LONG); - assertOutputType("isnull(y)", inspector, ExprType.LONG); - assertOutputType("isnull(z)", inspector, ExprType.LONG); - assertOutputType("notnull(x)", inspector, ExprType.LONG); - assertOutputType("notnull(y)", inspector, ExprType.LONG); - assertOutputType("notnull(z)", inspector, ExprType.LONG); + assertOutputType("nvl(x, 'foo')", inspector, ExpressionType.STRING); + assertOutputType("nvl(y, 1)", inspector, ExpressionType.LONG); + assertOutputType("nvl(y, 1.1)", inspector, ExpressionType.DOUBLE); + assertOutputType("nvl(z, 2.0)", inspector, ExpressionType.DOUBLE); + assertOutputType("nvl(y, 2.0)", inspector, ExpressionType.DOUBLE); + assertOutputType("isnull(x)", inspector, ExpressionType.LONG); + assertOutputType("isnull(y)", inspector, ExpressionType.LONG); + assertOutputType("isnull(z)", inspector, ExpressionType.LONG); + assertOutputType("notnull(x)", inspector, ExpressionType.LONG); + assertOutputType("notnull(y)", inspector, ExpressionType.LONG); + assertOutputType("notnull(z)", inspector, ExpressionType.LONG); } @Test public void testStringFunctions() { - assertOutputType("concat(x, 'foo')", inspector, ExprType.STRING); - assertOutputType("concat(y, 'foo')", inspector, ExprType.STRING); - assertOutputType("concat(z, 'foo')", inspector, ExprType.STRING); - - assertOutputType("strlen(x)", inspector, ExprType.LONG); - assertOutputType("format('%s', x)", inspector, ExprType.STRING); - assertOutputType("format('%s', y)", inspector, ExprType.STRING); - assertOutputType("format('%s', z)", inspector, ExprType.STRING); - assertOutputType("strpos(x, x_)", inspector, ExprType.LONG); - assertOutputType("strpos(x, y)", inspector, ExprType.LONG); - assertOutputType("strpos(x, z)", inspector, ExprType.LONG); - assertOutputType("substring(x, 1, 2)", inspector, ExprType.STRING); - assertOutputType("left(x, 1)", inspector, ExprType.STRING); - assertOutputType("right(x, 1)", inspector, ExprType.STRING); - assertOutputType("replace(x, 'foo', '')", inspector, ExprType.STRING); - assertOutputType("lower(x)", inspector, ExprType.STRING); - assertOutputType("upper(x)", inspector, ExprType.STRING); - assertOutputType("reverse(x)", inspector, ExprType.STRING); - assertOutputType("repeat(x, 4)", inspector, ExprType.STRING); + assertOutputType("concat(x, 'foo')", inspector, ExpressionType.STRING); + assertOutputType("concat(y, 'foo')", inspector, ExpressionType.STRING); + assertOutputType("concat(z, 'foo')", inspector, ExpressionType.STRING); + + assertOutputType("strlen(x)", inspector, ExpressionType.LONG); + assertOutputType("format('%s', x)", inspector, ExpressionType.STRING); + assertOutputType("format('%s', y)", inspector, ExpressionType.STRING); + assertOutputType("format('%s', z)", inspector, ExpressionType.STRING); + assertOutputType("strpos(x, x_)", inspector, ExpressionType.LONG); + assertOutputType("strpos(x, y)", inspector, ExpressionType.LONG); + assertOutputType("strpos(x, z)", inspector, ExpressionType.LONG); + assertOutputType("substring(x, 1, 2)", inspector, ExpressionType.STRING); + assertOutputType("left(x, 1)", inspector, ExpressionType.STRING); + assertOutputType("right(x, 1)", inspector, ExpressionType.STRING); + assertOutputType("replace(x, 'foo', '')", inspector, ExpressionType.STRING); + assertOutputType("lower(x)", inspector, ExpressionType.STRING); + assertOutputType("upper(x)", inspector, ExpressionType.STRING); + assertOutputType("reverse(x)", inspector, ExpressionType.STRING); + assertOutputType("repeat(x, 4)", inspector, ExpressionType.STRING); } @Test public void testArrayFunctions() { - assertOutputType("array(1, 2, 3)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array(1, 2, 3.0)", inspector, ExprType.DOUBLE_ARRAY); - - assertOutputType("array_length(a)", inspector, ExprType.LONG); - assertOutputType("array_length(b)", inspector, ExprType.LONG); - assertOutputType("array_length(c)", inspector, ExprType.LONG); - - assertOutputType("string_to_array(x, ',')", inspector, ExprType.STRING_ARRAY); - - assertOutputType("array_to_string(a, ',')", inspector, ExprType.STRING); - assertOutputType("array_to_string(b, ',')", inspector, ExprType.STRING); - assertOutputType("array_to_string(c, ',')", inspector, ExprType.STRING); - - assertOutputType("array_offset(a, 1)", inspector, ExprType.STRING); - assertOutputType("array_offset(b, 1)", inspector, ExprType.LONG); - assertOutputType("array_offset(c, 1)", inspector, ExprType.DOUBLE); - - assertOutputType("array_ordinal(a, 1)", inspector, ExprType.STRING); - assertOutputType("array_ordinal(b, 1)", inspector, ExprType.LONG); - assertOutputType("array_ordinal(c, 1)", inspector, ExprType.DOUBLE); - - assertOutputType("array_offset_of(a, 'a')", inspector, ExprType.LONG); - assertOutputType("array_offset_of(b, 1)", inspector, ExprType.LONG); - assertOutputType("array_offset_of(c, 1.0)", inspector, ExprType.LONG); - - assertOutputType("array_ordinal_of(a, 'a')", inspector, ExprType.LONG); - assertOutputType("array_ordinal_of(b, 1)", inspector, ExprType.LONG); - assertOutputType("array_ordinal_of(c, 1.0)", inspector, ExprType.LONG); - - assertOutputType("array_append(x, x_)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_append(a, x_)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_append(y, y_)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_append(b, y_)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_append(z, z_)", inspector, ExprType.DOUBLE_ARRAY); - assertOutputType("array_append(c, z_)", inspector, ExprType.DOUBLE_ARRAY); - - assertOutputType("array_concat(x, a)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_concat(a, a)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_concat(y, b)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_concat(b, b)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_concat(z, c)", inspector, ExprType.DOUBLE_ARRAY); - assertOutputType("array_concat(c, c)", inspector, ExprType.DOUBLE_ARRAY); - - assertOutputType("array_contains(a, 'a')", inspector, ExprType.LONG); - assertOutputType("array_contains(b, 1)", inspector, ExprType.LONG); - assertOutputType("array_contains(c, 2.0)", inspector, ExprType.LONG); - - assertOutputType("array_overlap(a, a)", inspector, ExprType.LONG); - assertOutputType("array_overlap(b, b)", inspector, ExprType.LONG); - assertOutputType("array_overlap(c, c)", inspector, ExprType.LONG); - - assertOutputType("array_slice(a, 1, 2)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_slice(b, 1, 2)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_slice(c, 1, 2)", inspector, ExprType.DOUBLE_ARRAY); - - assertOutputType("array_prepend(x, a)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_prepend(x, x_)", inspector, ExprType.STRING_ARRAY); - assertOutputType("array_prepend(y, b)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_prepend(y, y_)", inspector, ExprType.LONG_ARRAY); - assertOutputType("array_prepend(z, c)", inspector, ExprType.DOUBLE_ARRAY); - assertOutputType("array_prepend(z, z_)", inspector, ExprType.DOUBLE_ARRAY); + assertOutputType("array(1, 2, 3)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array(1, 2, 3.0)", inspector, ExpressionType.DOUBLE_ARRAY); + + assertOutputType("array_length(a)", inspector, ExpressionType.LONG); + assertOutputType("array_length(b)", inspector, ExpressionType.LONG); + assertOutputType("array_length(c)", inspector, ExpressionType.LONG); + + assertOutputType("string_to_array(x, ',')", inspector, ExpressionType.STRING_ARRAY); + + assertOutputType("array_to_string(a, ',')", inspector, ExpressionType.STRING); + assertOutputType("array_to_string(b, ',')", inspector, ExpressionType.STRING); + assertOutputType("array_to_string(c, ',')", inspector, ExpressionType.STRING); + + assertOutputType("array_offset(a, 1)", inspector, ExpressionType.STRING); + assertOutputType("array_offset(b, 1)", inspector, ExpressionType.LONG); + assertOutputType("array_offset(c, 1)", inspector, ExpressionType.DOUBLE); + + assertOutputType("array_ordinal(a, 1)", inspector, ExpressionType.STRING); + assertOutputType("array_ordinal(b, 1)", inspector, ExpressionType.LONG); + assertOutputType("array_ordinal(c, 1)", inspector, ExpressionType.DOUBLE); + + assertOutputType("array_offset_of(a, 'a')", inspector, ExpressionType.LONG); + assertOutputType("array_offset_of(b, 1)", inspector, ExpressionType.LONG); + assertOutputType("array_offset_of(c, 1.0)", inspector, ExpressionType.LONG); + + assertOutputType("array_ordinal_of(a, 'a')", inspector, ExpressionType.LONG); + assertOutputType("array_ordinal_of(b, 1)", inspector, ExpressionType.LONG); + assertOutputType("array_ordinal_of(c, 1.0)", inspector, ExpressionType.LONG); + + assertOutputType("array_append(x, x_)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_append(a, x_)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_append(y, y_)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_append(b, y_)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_append(z, z_)", inspector, ExpressionType.DOUBLE_ARRAY); + assertOutputType("array_append(c, z_)", inspector, ExpressionType.DOUBLE_ARRAY); + + assertOutputType("array_concat(x, a)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_concat(a, a)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_concat(y, b)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_concat(b, b)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_concat(z, c)", inspector, ExpressionType.DOUBLE_ARRAY); + assertOutputType("array_concat(c, c)", inspector, ExpressionType.DOUBLE_ARRAY); + + assertOutputType("array_contains(a, 'a')", inspector, ExpressionType.LONG); + assertOutputType("array_contains(b, 1)", inspector, ExpressionType.LONG); + assertOutputType("array_contains(c, 2.0)", inspector, ExpressionType.LONG); + + assertOutputType("array_overlap(a, a)", inspector, ExpressionType.LONG); + assertOutputType("array_overlap(b, b)", inspector, ExpressionType.LONG); + assertOutputType("array_overlap(c, c)", inspector, ExpressionType.LONG); + + assertOutputType("array_slice(a, 1, 2)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_slice(b, 1, 2)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_slice(c, 1, 2)", inspector, ExpressionType.DOUBLE_ARRAY); + + assertOutputType("array_prepend(x, a)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_prepend(x, x_)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("array_prepend(y, b)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_prepend(y, y_)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("array_prepend(z, c)", inspector, ExpressionType.DOUBLE_ARRAY); + assertOutputType("array_prepend(z, z_)", inspector, ExpressionType.DOUBLE_ARRAY); } @Test public void testReduceFunctions() { - assertOutputType("greatest('B', x, 'A')", inspector, ExprType.STRING); - assertOutputType("greatest(y, 0)", inspector, ExprType.LONG); - assertOutputType("greatest(34.0, z, 5.0, 767.0)", inspector, ExprType.DOUBLE); + assertOutputType("greatest('B', x, 'A')", inspector, ExpressionType.STRING); + assertOutputType("greatest(y, 0)", inspector, ExpressionType.LONG); + assertOutputType("greatest(34.0, z, 5.0, 767.0)", inspector, ExpressionType.DOUBLE); - assertOutputType("least('B', x, 'A')", inspector, ExprType.STRING); - assertOutputType("least(y, 0)", inspector, ExprType.LONG); - assertOutputType("least(34.0, z, 5.0, 767.0)", inspector, ExprType.DOUBLE); + assertOutputType("least('B', x, 'A')", inspector, ExpressionType.STRING); + assertOutputType("least(y, 0)", inspector, ExpressionType.LONG); + assertOutputType("least(34.0, z, 5.0, 767.0)", inspector, ExpressionType.DOUBLE); } @Test public void testApplyFunctions() { - assertOutputType("map((x) -> concat(x, 'foo'), x)", inspector, ExprType.STRING_ARRAY); - assertOutputType("map((x) -> x + x, y)", inspector, ExprType.LONG_ARRAY); - assertOutputType("map((x) -> x + x, z)", inspector, ExprType.DOUBLE_ARRAY); - assertOutputType("map((x) -> concat(x, 'foo'), a)", inspector, ExprType.STRING_ARRAY); - assertOutputType("map((x) -> x + x, b)", inspector, ExprType.LONG_ARRAY); - assertOutputType("map((x) -> x + x, c)", inspector, ExprType.DOUBLE_ARRAY); + assertOutputType("map((x) -> concat(x, 'foo'), x)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("map((x) -> x + x, y)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("map((x) -> x + x, z)", inspector, ExpressionType.DOUBLE_ARRAY); + assertOutputType("map((x) -> concat(x, 'foo'), a)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("map((x) -> x + x, b)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("map((x) -> x + x, c)", inspector, ExpressionType.DOUBLE_ARRAY); assertOutputType( "cartesian_map((x, y) -> concat(x, y), ['foo', 'bar', 'baz', 'foobar'], ['bar', 'baz'])", inspector, - ExprType.STRING_ARRAY + ExpressionType.STRING_ARRAY ); - assertOutputType("fold((x, acc) -> x + acc, y, 0)", inspector, ExprType.LONG); - assertOutputType("fold((x, acc) -> x + acc, y, y)", inspector, ExprType.LONG); - assertOutputType("fold((x, acc) -> x + acc, y, 1.0)", inspector, ExprType.DOUBLE); - assertOutputType("fold((x, acc) -> x + acc, y, z)", inspector, ExprType.DOUBLE); - - assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 0)", inspector, ExprType.LONG); - assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, y)", inspector, ExprType.LONG); - assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 1.0)", inspector, ExprType.DOUBLE); - assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, z)", inspector, ExprType.DOUBLE); - - assertOutputType("filter((x) -> x == 'foo', a)", inspector, ExprType.STRING_ARRAY); - assertOutputType("filter((x) -> x > 1, b)", inspector, ExprType.LONG_ARRAY); - assertOutputType("filter((x) -> x > 1, c)", inspector, ExprType.DOUBLE_ARRAY); - - assertOutputType("any((x) -> x == 'foo', a)", inspector, ExprType.LONG); - assertOutputType("any((x) -> x > 1, b)", inspector, ExprType.LONG); - assertOutputType("any((x) -> x > 1.2, c)", inspector, ExprType.LONG); - - assertOutputType("all((x) -> x == 'foo', a)", inspector, ExprType.LONG); - assertOutputType("all((x) -> x > 1, b)", inspector, ExprType.LONG); - assertOutputType("all((x) -> x > 1.2, c)", inspector, ExprType.LONG); + assertOutputType("fold((x, acc) -> x + acc, y, 0)", inspector, ExpressionType.LONG); + assertOutputType("fold((x, acc) -> x + acc, y, y)", inspector, ExpressionType.LONG); + assertOutputType("fold((x, acc) -> x + acc, y, 1.0)", inspector, ExpressionType.DOUBLE); + assertOutputType("fold((x, acc) -> x + acc, y, z)", inspector, ExpressionType.DOUBLE); + + assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 0)", inspector, ExpressionType.LONG); + assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, y)", inspector, ExpressionType.LONG); + assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, 1.0)", inspector, ExpressionType.DOUBLE); + assertOutputType("cartesian_fold((x, y, acc) -> x + y + acc, y, z, z)", inspector, ExpressionType.DOUBLE); + + assertOutputType("filter((x) -> x == 'foo', a)", inspector, ExpressionType.STRING_ARRAY); + assertOutputType("filter((x) -> x > 1, b)", inspector, ExpressionType.LONG_ARRAY); + assertOutputType("filter((x) -> x > 1, c)", inspector, ExpressionType.DOUBLE_ARRAY); + + assertOutputType("any((x) -> x == 'foo', a)", inspector, ExpressionType.LONG); + assertOutputType("any((x) -> x > 1, b)", inspector, ExpressionType.LONG); + assertOutputType("any((x) -> x > 1.2, c)", inspector, ExpressionType.LONG); + + assertOutputType("all((x) -> x == 'foo', a)", inspector, ExpressionType.LONG); + assertOutputType("all((x) -> x > 1, b)", inspector, ExpressionType.LONG); + assertOutputType("all((x) -> x > 1.2, c)", inspector, ExpressionType.LONG); } @@ -399,63 +399,63 @@ public void testEvalAutoConversion() final ExprEval arrayEval = ExprEval.ofLongArray(new Long[]{1L, 2L, 3L}); // only long stays long - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(longEval, longEval)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.autoDetect(longEval, longEval)); // only string stays string - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.autoDetect(nullStringEval, nullStringEval)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.autoDetect(stringEval, stringEval)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.autoDetect(nullStringEval, nullStringEval)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.autoDetect(stringEval, stringEval)); // if only 1 argument is a string, preserve the other type - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(nullStringEval, longEval)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.autoDetect(longEval, nullStringEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, nullStringEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(nullStringEval, doubleEval)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.autoDetect(nullStringEval, longEval)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.autoDetect(longEval, nullStringEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(doubleEval, nullStringEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(nullStringEval, doubleEval)); // for operators, doubles is the catch all - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, doubleEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, longEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, doubleEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(longEval, doubleEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(doubleEval, longEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(doubleEval, doubleEval)); // ... even when non-null strings are used with non-double types - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, stringEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, stringEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(stringEval, doubleEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(longEval, stringEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(doubleEval, stringEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(stringEval, doubleEval)); // arrays are not a good idea to use with this method - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, nullStringEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, doubleEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(arrayEval, longEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(nullStringEval, arrayEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(doubleEval, arrayEval)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.autoDetect(longEval, arrayEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(arrayEval, nullStringEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(arrayEval, doubleEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(arrayEval, longEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(nullStringEval, arrayEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(doubleEval, arrayEval)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.autoDetect(longEval, arrayEval)); } @Test public void testOperatorAutoConversion() { // nulls output other - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(ExprType.LONG, null)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(null, ExprType.LONG)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.DOUBLE, null)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(null, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.operator(ExprType.STRING, null)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.operator(null, ExprType.STRING)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.operator(ExpressionType.LONG, null)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.operator(null, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.DOUBLE, null)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(null, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.operator(ExpressionType.STRING, null)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.operator(null, ExpressionType.STRING)); // only long stays long - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.operator(ExprType.LONG, ExprType.LONG)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.operator(ExpressionType.LONG, ExpressionType.LONG)); // only string stays string - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.operator(ExprType.STRING, ExprType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.operator(ExpressionType.STRING, ExpressionType.STRING)); // for operators, doubles is the catch all - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.LONG, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.DOUBLE, ExprType.LONG)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.DOUBLE, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.DOUBLE, ExprType.STRING)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.STRING, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.STRING, ExprType.LONG)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.operator(ExprType.LONG, ExprType.STRING)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.LONG, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.DOUBLE, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.DOUBLE, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.DOUBLE, ExpressionType.STRING)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.STRING, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.STRING, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.operator(ExpressionType.LONG, ExpressionType.STRING)); // unless it is an array, and those have to be the same - Assert.assertEquals(ExprType.LONG_ARRAY, ExprTypeConversion.operator(ExprType.LONG_ARRAY, ExprType.LONG_ARRAY)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, ExpressionTypeConversion.operator(ExpressionType.LONG_ARRAY, ExpressionType.LONG_ARRAY)); Assert.assertEquals( - ExprType.DOUBLE_ARRAY, - ExprTypeConversion.operator(ExprType.DOUBLE_ARRAY, ExprType.DOUBLE_ARRAY) + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.DOUBLE_ARRAY, ExpressionType.DOUBLE_ARRAY) ); Assert.assertEquals( - ExprType.STRING_ARRAY, - ExprTypeConversion.operator(ExprType.STRING_ARRAY, ExprType.STRING_ARRAY) + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.operator(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); } @@ -463,33 +463,33 @@ public void testOperatorAutoConversion() public void testFunctionAutoConversion() { // nulls output other - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(ExprType.LONG, null)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(null, ExprType.LONG)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, null)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(null, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, null)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(null, ExprType.STRING)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.function(ExpressionType.LONG, null)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.function(null, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.function(ExpressionType.DOUBLE, null)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.function(null, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.STRING, null)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(null, ExpressionType.STRING)); // only long stays long - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.function(ExprType.LONG, ExprType.LONG)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.function(ExpressionType.LONG, ExpressionType.LONG)); // any double makes all doubles - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.LONG, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.LONG)); - Assert.assertEquals(ExprType.DOUBLE, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.DOUBLE)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.function(ExpressionType.LONG, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.function(ExpressionType.DOUBLE, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.DOUBLE, ExpressionTypeConversion.function(ExpressionType.DOUBLE, ExpressionType.DOUBLE)); // any string makes become string - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.LONG, ExprType.STRING)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.LONG)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.DOUBLE, ExprType.STRING)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.function(ExprType.STRING, ExprType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.LONG, ExpressionType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.STRING, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.DOUBLE, ExpressionType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.STRING, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.function(ExpressionType.STRING, ExpressionType.STRING)); // unless it is an array, and those have to be the same - Assert.assertEquals(ExprType.LONG_ARRAY, ExprTypeConversion.function(ExprType.LONG_ARRAY, ExprType.LONG_ARRAY)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, ExpressionTypeConversion.function(ExpressionType.LONG_ARRAY, ExpressionType.LONG_ARRAY)); Assert.assertEquals( - ExprType.DOUBLE_ARRAY, - ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.DOUBLE_ARRAY) + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.DOUBLE_ARRAY) ); Assert.assertEquals( - ExprType.STRING_ARRAY, - ExprTypeConversion.function(ExprType.STRING_ARRAY, ExprType.STRING_ARRAY) + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.function(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); } @@ -497,32 +497,32 @@ public void testFunctionAutoConversion() public void testIntegerFunctionAutoConversion() { // nulls output other - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, null)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(null, ExprType.LONG)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, null)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(null, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.STRING, null)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(null, ExprType.STRING)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.LONG, null)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(null, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.DOUBLE, null)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(null, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.STRING, null)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(null, ExpressionType.STRING)); // all numbers are longs - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, ExprType.LONG)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.LONG, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, ExprType.LONG)); - Assert.assertEquals(ExprType.LONG, ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, ExprType.DOUBLE)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.LONG, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.LONG, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.DOUBLE, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.LONG, ExpressionTypeConversion.integerMathFunction(ExpressionType.DOUBLE, ExpressionType.DOUBLE)); // any string makes become string - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.LONG, ExprType.STRING)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.STRING, ExprType.LONG)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.DOUBLE, ExprType.STRING)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.STRING, ExprType.DOUBLE)); - Assert.assertEquals(ExprType.STRING, ExprTypeConversion.integerMathFunction(ExprType.STRING, ExprType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.LONG, ExpressionType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.STRING, ExpressionType.LONG)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.DOUBLE, ExpressionType.STRING)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.STRING, ExpressionType.DOUBLE)); + Assert.assertEquals(ExpressionType.STRING, ExpressionTypeConversion.integerMathFunction(ExpressionType.STRING, ExpressionType.STRING)); // unless it is an array - Assert.assertEquals(ExprType.LONG_ARRAY, ExprTypeConversion.integerMathFunction(ExprType.LONG_ARRAY, ExprType.LONG_ARRAY)); + Assert.assertEquals(ExpressionType.LONG_ARRAY, ExpressionTypeConversion.integerMathFunction(ExpressionType.LONG_ARRAY, ExpressionType.LONG_ARRAY)); Assert.assertEquals( - ExprType.DOUBLE_ARRAY, - ExprTypeConversion.integerMathFunction(ExprType.DOUBLE_ARRAY, ExprType.DOUBLE_ARRAY) + ExpressionType.DOUBLE_ARRAY, + ExpressionTypeConversion.integerMathFunction(ExpressionType.DOUBLE_ARRAY, ExpressionType.DOUBLE_ARRAY) ); Assert.assertEquals( - ExprType.STRING_ARRAY, - ExprTypeConversion.integerMathFunction(ExprType.STRING_ARRAY, ExprType.STRING_ARRAY) + ExpressionType.STRING_ARRAY, + ExpressionTypeConversion.integerMathFunction(ExpressionType.STRING_ARRAY, ExpressionType.STRING_ARRAY) ); } @@ -530,31 +530,31 @@ public void testIntegerFunctionAutoConversion() public void testAutoConversionArrayMismatchArrays() { expectedException.expect(IAE.class); - ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.LONG_ARRAY); + ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.LONG_ARRAY); } @Test public void testAutoConversionArrayMismatchArrayScalar() { expectedException.expect(IAE.class); - ExprTypeConversion.function(ExprType.DOUBLE_ARRAY, ExprType.LONG); + ExpressionTypeConversion.function(ExpressionType.DOUBLE_ARRAY, ExpressionType.LONG); } @Test public void testAutoConversionArrayMismatchScalarArray() { expectedException.expect(IAE.class); - ExprTypeConversion.function(ExprType.DOUBLE, ExprType.LONG_ARRAY); + ExpressionTypeConversion.function(ExpressionType.DOUBLE, ExpressionType.LONG_ARRAY); } - private void assertOutputType(String expression, Expr.InputBindingInspector inspector, ExprType outputType) + private void assertOutputType(String expression, Expr.InputBindingInspector inspector, ExpressionType outputType) { final Expr expr = Parser.parse(expression, ExprMacroTable.nil(), false); Assert.assertEquals(outputType, expr.getOutputType(inspector)); } - Expr.InputBindingInspector inspectorFromMap(Map types) + Expr.InputBindingInspector inspectorFromMap(Map types) { - return types::get; + return key -> types.get(key); } } diff --git a/core/src/test/java/org/apache/druid/math/expr/ParserTest.java b/core/src/test/java/org/apache/druid/math/expr/ParserTest.java index 12dcfee0b894..82405cb8434f 100644 --- a/core/src/test/java/org/apache/druid/math/expr/ParserTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/ParserTest.java @@ -56,25 +56,25 @@ public void testSimple() public void testParseConstants() { validateLiteral("null", null, null); - validateLiteral("'hello'", ExprType.STRING, "hello"); - validateLiteral("'hello \\uD83E\\uDD18'", ExprType.STRING, "hello \uD83E\uDD18"); - validateLiteral("1", ExprType.LONG, 1L); - validateLiteral("1.", ExprType.DOUBLE, 1.0, false); - validateLiteral("1.234", ExprType.DOUBLE, 1.234); - validateLiteral("1e10", ExprType.DOUBLE, 1.0E10, false); - validateLiteral("1e-10", ExprType.DOUBLE, 1.0E-10, false); - validateLiteral("1E10", ExprType.DOUBLE, 1.0E10, false); - validateLiteral("1E-10", ExprType.DOUBLE, 1.0E-10, false); - validateLiteral("1.E10", ExprType.DOUBLE, 1.0E10, false); - validateLiteral("1.E-10", ExprType.DOUBLE, 1.0E-10, false); - validateLiteral("1.e10", ExprType.DOUBLE, 1.0E10, false); - validateLiteral("1.e-10", ExprType.DOUBLE, 1.0E-10, false); - validateLiteral("1.1e10", ExprType.DOUBLE, 1.1E10, false); - validateLiteral("1.1e-10", ExprType.DOUBLE, 1.1E-10, false); - validateLiteral("1.1E10", ExprType.DOUBLE, 1.1E10); - validateLiteral("1.1E-10", ExprType.DOUBLE, 1.1E-10); - validateLiteral("Infinity", ExprType.DOUBLE, Double.POSITIVE_INFINITY); - validateLiteral("NaN", ExprType.DOUBLE, Double.NaN); + validateLiteral("'hello'", ExpressionType.STRING, "hello"); + validateLiteral("'hello \\uD83E\\uDD18'", ExpressionType.STRING, "hello \uD83E\uDD18"); + validateLiteral("1", ExpressionType.LONG, 1L); + validateLiteral("1.", ExpressionType.DOUBLE, 1.0, false); + validateLiteral("1.234", ExpressionType.DOUBLE, 1.234); + validateLiteral("1e10", ExpressionType.DOUBLE, 1.0E10, false); + validateLiteral("1e-10", ExpressionType.DOUBLE, 1.0E-10, false); + validateLiteral("1E10", ExpressionType.DOUBLE, 1.0E10, false); + validateLiteral("1E-10", ExpressionType.DOUBLE, 1.0E-10, false); + validateLiteral("1.E10", ExpressionType.DOUBLE, 1.0E10, false); + validateLiteral("1.E-10", ExpressionType.DOUBLE, 1.0E-10, false); + validateLiteral("1.e10", ExpressionType.DOUBLE, 1.0E10, false); + validateLiteral("1.e-10", ExpressionType.DOUBLE, 1.0E-10, false); + validateLiteral("1.1e10", ExpressionType.DOUBLE, 1.1E10, false); + validateLiteral("1.1e-10", ExpressionType.DOUBLE, 1.1E-10, false); + validateLiteral("1.1E10", ExpressionType.DOUBLE, 1.1E10); + validateLiteral("1.1E-10", ExpressionType.DOUBLE, 1.1E-10); + validateLiteral("Infinity", ExpressionType.DOUBLE, Double.POSITIVE_INFINITY); + validateLiteral("NaN", ExpressionType.DOUBLE, Double.NaN); } @Test @@ -612,12 +612,12 @@ public void testUniquify() ); } - private void validateLiteral(String expr, ExprType type, Object expected) + private void validateLiteral(String expr, ExpressionType type, Object expected) { validateLiteral(expr, type, expected, true); } - private void validateLiteral(String expr, ExprType type, Object expected, boolean roundTrip) + private void validateLiteral(String expr, ExpressionType type, Object expected, boolean roundTrip) { Expr parsed = Parser.parse(expr, ExprMacroTable.nil(), false); Expr parsedFlat = Parser.parse(expr, ExprMacroTable.nil(), true); diff --git a/core/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java b/core/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java index 7b005ee60d67..81abed544c10 100644 --- a/core/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/VectorExprSanityTest.java @@ -51,13 +51,13 @@ public class VectorExprSanityTest extends InitializedNullHandlingTest private static final int NUM_ITERATIONS = 10; private static final int VECTOR_SIZE = 512; - final Map types = ImmutableMap.builder() - .put("l1", ExprType.LONG) - .put("l2", ExprType.LONG) - .put("d1", ExprType.DOUBLE) - .put("d2", ExprType.DOUBLE) - .put("s1", ExprType.STRING) - .put("s2", ExprType.STRING) + final Map types = ImmutableMap.builder() + .put("l1", ExpressionType.LONG) + .put("l2", ExpressionType.LONG) + .put("d1", ExpressionType.DOUBLE) + .put("d2", ExpressionType.DOUBLE) + .put("s1", ExpressionType.STRING) + .put("s2", ExpressionType.STRING) .build(); @Test @@ -211,7 +211,7 @@ public void testStringFns() testExpression("concat(s1,'-',s2,'-',l1,'-',d1)", types); } - static void testFunctions(Map types, String[] templates, String[] args) + static void testFunctions(Map types, String[] templates, String[] args) { for (String template : templates) { for (String arg : args) { @@ -221,7 +221,7 @@ static void testFunctions(Map types, String[] templates, Strin } } - static void testFunctions(Map types, String[] templates, String[][] argsArrays) + static void testFunctions(Map types, String[] templates, String[][] argsArrays) { for (String template : templates) { for (Object[] args : argsArrays) { @@ -231,7 +231,7 @@ static void testFunctions(Map types, String[] templates, Strin } } - static void testExpression(String expr, Map types) + static void testExpression(String expr, Map types) { log.debug("[%s]", expr); Expr parsed = Parser.parse(expr, ExprMacroTable.nil()); @@ -252,7 +252,7 @@ private static void testExpressionWithBindings( ) { Assert.assertTrue(StringUtils.format("Cannot vectorize %s", expr), parsed.canVectorize(bindings.rhs)); - ExprType outputType = parsed.getOutputType(bindings.rhs); + ExpressionType outputType = parsed.getOutputType(bindings.rhs); ExprEvalVector vectorEval = parsed.buildVectorized(bindings.rhs).evalVector(bindings.rhs); // 'null' expressions can have an output type of null, but still evaluate in default mode, so skip type checks if (outputType != null) { @@ -274,7 +274,7 @@ private static void testExpressionWithBindings( static NonnullPair makeRandomizedBindings( int vectorSize, - Map types + Map types ) { @@ -291,7 +291,7 @@ static NonnullPair makeRandomized static NonnullPair makeSequentialBinding( int vectorSize, - Map types + Map types ) { @@ -334,7 +334,7 @@ public String get() static NonnullPair makeBindings( int vectorSize, - Map types, + Map types, LongSupplier longsFn, DoubleSupplier doublesFn, BooleanSupplier nullsFn, @@ -345,10 +345,10 @@ static NonnullPair makeBindings( SettableObjectBinding[] objectBindings = new SettableObjectBinding[vectorSize]; final boolean hasNulls = NullHandling.sqlCompatible(); - for (Map.Entry entry : types.entrySet()) { + for (Map.Entry entry : types.entrySet()) { boolean[] nulls = new boolean[vectorSize]; - switch (entry.getValue()) { + switch (entry.getValue().getType()) { case LONG: long[] longs = new long[vectorSize]; for (int i = 0; i < vectorSize; i++) { @@ -421,7 +421,7 @@ static class SettableVectorInputBinding implements Expr.VectorInputBinding private final Map longs; private final Map doubles; private final Map objects; - private final Map types; + private final Map types; private final int vectorSize; @@ -437,7 +437,7 @@ static class SettableVectorInputBinding implements Expr.VectorInputBinding this.vectorSize = vectorSize; } - public SettableVectorInputBinding addBinding(String name, ExprType type, boolean[] nulls) + public SettableVectorInputBinding addBinding(String name, ExpressionType type, boolean[] nulls) { this.nulls.put(name, nulls); this.types.put(name, type); @@ -453,7 +453,7 @@ public SettableVectorInputBinding addLong(String name, long[] longs, boolean[] n { assert longs.length == vectorSize; this.longs.put(name, longs); - return addBinding(name, ExprType.LONG, nulls); + return addBinding(name, ExpressionType.LONG, nulls); } public SettableVectorInputBinding addDouble(String name, double[] doubles) @@ -465,14 +465,14 @@ public SettableVectorInputBinding addDouble(String name, double[] doubles, boole { assert doubles.length == vectorSize; this.doubles.put(name, doubles); - return addBinding(name, ExprType.DOUBLE, nulls); + return addBinding(name, ExpressionType.DOUBLE, nulls); } public SettableVectorInputBinding addString(String name, String[] strings) { assert strings.length == vectorSize; this.objects.put(name, strings); - return addBinding(name, ExprType.STRING, new boolean[strings.length]); + return addBinding(name, ExpressionType.STRING, new boolean[strings.length]); } @Override @@ -482,7 +482,7 @@ public T[] getObjectVector(String name) } @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { return types.get(name); } diff --git a/core/src/test/java/org/apache/druid/segment/column/ColumnTypeTest.java b/core/src/test/java/org/apache/druid/segment/column/ColumnTypeTest.java new file mode 100644 index 000000000000..4c192b2fae23 --- /dev/null +++ b/core/src/test/java/org/apache/druid/segment/column/ColumnTypeTest.java @@ -0,0 +1,159 @@ +/* + * 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.column; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +public class ColumnTypeTest +{ + public static final ColumnType SOME_COMPLEX = new ColumnType(ValueType.COMPLEX, "foo", null); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + Assert.assertEquals(ColumnType.STRING, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.STRING), ColumnType.class)); + Assert.assertEquals(ColumnType.LONG, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.LONG), ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.DOUBLE), ColumnType.class)); + Assert.assertEquals(ColumnType.FLOAT, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.FLOAT), ColumnType.class)); + Assert.assertEquals(ColumnType.STRING_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.STRING_ARRAY), ColumnType.class)); + Assert.assertEquals(ColumnType.LONG_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.LONG_ARRAY), ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.DOUBLE_ARRAY), ColumnType.class)); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, MAPPER.readValue(MAPPER.writeValueAsString(ColumnType.UNKNOWN_COMPLEX), ColumnType.class)); + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue(MAPPER.writeValueAsString(SOME_COMPLEX), ColumnType.class)); + } + + @Test + public void testSerdeLegacy() throws JsonProcessingException + { + Assert.assertEquals(ColumnType.STRING, MAPPER.readValue("\"STRING\"", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG, MAPPER.readValue("\"LONG\"", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE, MAPPER.readValue("\"DOUBLE\"", ColumnType.class)); + Assert.assertEquals(ColumnType.STRING_ARRAY, MAPPER.readValue("\"ARRAY\"", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG_ARRAY, MAPPER.readValue("\"ARRAY\"", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue("\"ARRAY\"", ColumnType.class)); + + ColumnType whatHaveIdone = new ColumnType(ValueType.ARRAY, null, new ColumnType(ValueType.ARRAY, null, SOME_COMPLEX)); + Assert.assertEquals(whatHaveIdone, MAPPER.readValue("\"ARRAY>>\"", ColumnType.class)); + + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue("\"COMPLEX\"", ColumnType.class)); + // make sure legacy works too + Assert.assertEquals(ColumnType.STRING, MAPPER.readValue("\"string\"", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG, MAPPER.readValue("\"long\"", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE, MAPPER.readValue("\"double\"", ColumnType.class)); + Assert.assertEquals(ColumnType.STRING_ARRAY, MAPPER.readValue("\"STRING_ARRAY\"", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG_ARRAY, MAPPER.readValue("\"LONG_ARRAY\"", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue("\"DOUBLE_ARRAY\"", ColumnType.class)); + Assert.assertEquals(ColumnType.STRING_ARRAY, MAPPER.readValue("\"string_array\"", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG_ARRAY, MAPPER.readValue("\"long_array\"", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue("\"double_array\"", ColumnType.class)); + // ARRAY<*> and COMPLEX<*> patterns must match exactly ... + Assert.assertNotEquals(ColumnType.STRING_ARRAY, MAPPER.readValue("\"array\"", ColumnType.class)); + Assert.assertNotEquals(ColumnType.LONG_ARRAY, MAPPER.readValue("\"array\"", ColumnType.class)); + Assert.assertNotEquals(SOME_COMPLEX, MAPPER.readValue("\"COMPLEX\"", ColumnType.class)); + // this works though because array recursively calls on element type... + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue("\"ARRAY\"", ColumnType.class)); + } + + @Test + public void testFutureProof() throws JsonProcessingException + { + // in case we ever want to switch from string serde to JSON objects for type info, be ready + Assert.assertEquals(ColumnType.STRING, MAPPER.readValue("{\"type\":\"STRING\"}", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG, MAPPER.readValue("{\"type\":\"LONG\"}", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE, MAPPER.readValue("{\"type\":\"DOUBLE\"}", ColumnType.class)); + Assert.assertEquals(ColumnType.STRING_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"STRING\"}}", ColumnType.class)); + Assert.assertEquals(ColumnType.LONG_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"LONG\"}}", ColumnType.class)); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"DOUBLE\"}}", ColumnType.class)); + + Assert.assertEquals(SOME_COMPLEX, MAPPER.readValue("{\"type\":\"COMPLEX\", \"complexTypeName\":\"foo\"}", ColumnType.class)); + + ColumnType whatHaveIdone = new ColumnType(ValueType.ARRAY, null, new ColumnType(ValueType.ARRAY, null, SOME_COMPLEX)); + Assert.assertEquals(whatHaveIdone, MAPPER.readValue("{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"ARRAY\", \"elementType\":{\"type\":\"COMPLEX\", \"complexTypeName\":\"foo\"}}}", ColumnType.class)); + } + + @Test + public void testFactoryFromOtherTypeSignatures() + { + Assert.assertEquals(ColumnType.LONG, ColumnTypeFactory.ofType(new SomeOtherTypeSignature(ValueType.LONG, null, null))); + Assert.assertEquals(ColumnType.LONG, ColumnTypeFactory.ofValueType(ValueType.LONG)); + Assert.assertEquals(ColumnType.FLOAT, ColumnTypeFactory.ofType(new SomeOtherTypeSignature(ValueType.FLOAT, null, null))); + Assert.assertEquals(ColumnType.FLOAT, ColumnTypeFactory.ofValueType(ValueType.FLOAT)); + Assert.assertEquals(ColumnType.DOUBLE, ColumnTypeFactory.ofType(new SomeOtherTypeSignature(ValueType.DOUBLE, null, null))); + Assert.assertEquals(ColumnType.DOUBLE, ColumnTypeFactory.ofValueType(ValueType.DOUBLE)); + Assert.assertEquals(ColumnType.STRING, ColumnTypeFactory.ofType(new SomeOtherTypeSignature(ValueType.STRING, null, null))); + Assert.assertEquals(ColumnType.STRING, ColumnTypeFactory.ofValueType(ValueType.STRING)); + Assert.assertEquals( + ColumnType.LONG_ARRAY, + ColumnTypeFactory.ofType( + new SomeOtherTypeSignature( + ValueType.ARRAY, + null, + new SomeOtherTypeSignature(ValueType.LONG, null, null) + ) + ) + ); + Assert.assertEquals( + ColumnType.DOUBLE_ARRAY, + ColumnTypeFactory.ofType( + new SomeOtherTypeSignature( + ValueType.ARRAY, + null, + new SomeOtherTypeSignature(ValueType.DOUBLE, null, null) + ) + ) + ); + Assert.assertEquals( + ColumnType.STRING_ARRAY, + ColumnTypeFactory.ofType( + new SomeOtherTypeSignature( + ValueType.ARRAY, + null, + new SomeOtherTypeSignature(ValueType.STRING, null, null) + ) + ) + ); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, ColumnTypeFactory.ofType(new SomeOtherTypeSignature(ValueType.COMPLEX, null, null))); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, ColumnTypeFactory.ofValueType(ValueType.COMPLEX)); + Assert.assertEquals( + SOME_COMPLEX, + ColumnTypeFactory.ofType( + new SomeOtherTypeSignature(ValueType.COMPLEX, SOME_COMPLEX.getComplexTypeName(), null) + ) + ); + } + + static class SomeOtherTypeSignature extends BaseTypeSignature + { + public SomeOtherTypeSignature( + ValueType valueType, + @Nullable String complexTypeName, + @Nullable TypeSignature elementType + ) + { + super(valueType, complexTypeName, elementType); + } + } +} diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java index bf6676569b56..8e083d6eb1ac 100644 --- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java +++ b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -194,27 +194,21 @@ public byte[] getCacheKey() .array(); } - @Override - public String getComplexTypeName() - { - return "distinctCount"; - } - /** * this aggregator only works on a single segment, so even though it stores a * {@link org.apache.druid.collections.bitmap.MutableBitmap} while computing, this value never leaves the aggregator * and {@link DistinctCountAggregator#get} returns an integer for the number of set bits in the bitmap. */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java index e769302c6939..01b694381038 100644 --- a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java +++ b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactory.java @@ -32,7 +32,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Collections; @@ -65,6 +65,7 @@ public class MomentSketchAggregatorFactory extends AggregatorFactory private final byte cacheTypeId; public static final String TYPE_NAME = "momentSketch"; + public static final ColumnType TYPE = ColumnType.ofComplex(TYPE_NAME); @JsonCreator public MomentSketchAggregatorFactory( @@ -108,7 +109,7 @@ public byte[] getCacheKey() public Aggregator factorize(ColumnSelectorFactory metricFactory) { ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName); - if (cap == null || ValueType.isNumeric(cap.getType())) { + if (cap == null || cap.isNumeric()) { return new MomentSketchBuildAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress()); } else { return new MomentSketchMergeAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress()); @@ -119,7 +120,7 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName); - if (cap == null || ValueType.isNumeric(cap.getType())) { + if (cap == null || cap.isNumeric()) { return new MomentSketchBuildBufferAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress()); } else { return new MomentSketchMergeBufferAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress()); @@ -239,25 +240,19 @@ public List requiredFields() return Collections.singletonList(fieldName); } - @Override - public String getComplexTypeName() - { - return TYPE_NAME; - } - /** * actual type is {@link MomentSketchWrapper} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return TYPE; } @Override diff --git a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMaxPostAggregator.java b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMaxPostAggregator.java index dfe3d3e8ec95..9237041598a1 100644 --- a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMaxPostAggregator.java +++ b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMaxPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -59,9 +59,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMinPostAggregator.java b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMinPostAggregator.java index b089a3d6c03b..34f3af05a47b 100644 --- a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMinPostAggregator.java +++ b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchMinPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -58,9 +58,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchQuantilePostAggregator.java b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchQuantilePostAggregator.java index 74d9fcbbad31..3852e788834c 100644 --- a/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchQuantilePostAggregator.java +++ b/extensions-contrib/momentsketch/src/main/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchQuantilePostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -63,9 +63,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactoryTest.java b/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactoryTest.java index 7b69d06054d9..5ad88eef3346 100644 --- a/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactoryTest.java +++ b/extensions-contrib/momentsketch/src/test/java/org/apache/druid/query/aggregation/momentsketch/aggregator/MomentSketchAggregatorFactoryTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -75,13 +75,13 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("moment", ValueType.COMPLEX) - .add("momentMerge", ValueType.COMPLEX) - .add("moment-access", ValueType.COMPLEX) - .add("moment-finalize", ValueType.COMPLEX) - .add("momentMerge-access", ValueType.COMPLEX) - .add("momentMerge-finalize", ValueType.COMPLEX) + .add("count", ColumnType.LONG) + .add("moment", MomentSketchAggregatorFactory.TYPE) + .add("momentMerge", MomentSketchAggregatorFactory.TYPE) + .add("moment-access", MomentSketchAggregatorFactory.TYPE) + .add("moment-finalize", MomentSketchAggregatorFactory.TYPE) + .add("momentMerge-access", MomentSketchAggregatorFactory.TYPE) + .add("momentMerge-finalize", MomentSketchAggregatorFactory.TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java index 8467bb97ed00..5b7b2e8b8b7c 100644 --- a/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java +++ b/extensions-contrib/moving-average-query/src/main/java/org/apache/druid/query/movingaverage/AveragerFactoryWrapper.java @@ -24,7 +24,7 @@ import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.movingaverage.averagers.AveragerFactory; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -161,25 +161,14 @@ public byte[] getCacheKey() throw new UnsupportedOperationException("Invalid operation for AveragerFactoryWrapper."); } - /** - * This method must be implemented since it is called by - * {@link org.apache.druid.query.groupby.GroupByQuery#computeResultRowSignature}. Returning "COMPLEX" causes the - * return type to be treated as unknown. - */ - @Override - public String getComplexTypeName() - { - return ValueType.COMPLEX.name(); - } - @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return ColumnType.UNKNOWN_COMPLEX; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return getType(); } diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java index c63c193f7876..ed1020ecb238 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -75,6 +75,7 @@ public class TDigestSketchAggregatorFactory extends AggregatorFactory private final byte cacheTypeId; public static final String TYPE_NAME = "tDigestSketch"; + public static final ColumnType TYPE = ColumnType.ofComplex(TYPE_NAME); @JsonCreator public TDigestSketchAggregatorFactory( @@ -211,25 +212,19 @@ public List requiredFields() return Collections.singletonList(fieldName); } - @Override - public String getComplexTypeName() - { - return TYPE_NAME; - } - /** * actual type is {@link MergingDigest} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return TYPE; } @Override diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilePostAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilePostAggregator.java index f1bd8fa832dc..987a15a7db08 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilePostAggregator.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilePostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -68,9 +68,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilesPostAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilesPostAggregator.java index 122969df6834..25f4c2500743 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilesPostAggregator.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchToQuantilesPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -69,9 +69,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java index ee89702f9c88..34224a1ce1ea 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestGenerateSketchSqlAggregator.java @@ -36,8 +36,8 @@ import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchAggregatorFactory; import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchUtils; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -138,7 +138,7 @@ public Aggregation toDruidAggregation( VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( plannerContext, input, - ValueType.FLOAT + ColumnType.FLOAT ); aggregatorFactory = new TDigestSketchAggregatorFactory( aggName, diff --git a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java index 314edfa30a85..f15130b83793 100644 --- a/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java +++ b/extensions-contrib/tdigestsketch/src/main/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchQuantileSqlAggregator.java @@ -39,8 +39,8 @@ import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchToQuantilePostAggregator; import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchUtils; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -157,7 +157,7 @@ public Aggregation toDruidAggregation( VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( plannerContext, input, - ValueType.FLOAT + ColumnType.FLOAT ); aggregatorFactory = new TDigestSketchAggregatorFactory( sketchName, diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactoryTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactoryTest.java index cb08f67c875a..be3a80290237 100644 --- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactoryTest.java +++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/TDigestSketchAggregatorFactoryTest.java @@ -26,8 +26,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -54,10 +54,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("tdigest", ValueType.COMPLEX) - .add("tdigest-access", ValueType.COMPLEX) - .add("tdigest-finalize", ValueType.COMPLEX) + .add("count", ColumnType.LONG) + .add("tdigest", TDigestSketchAggregatorFactory.TYPE) + .add("tdigest-access", TDigestSketchAggregatorFactory.TYPE) + .add("tdigest-finalize", TDigestSketchAggregatorFactory.TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java index c1737f903274..4f3886aea86c 100644 --- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java +++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java @@ -41,7 +41,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -158,7 +158,7 @@ public void testComputingSketchOnCastedString() throws Exception new ExpressionVirtualColumn( "v0", "CAST(\"dim1\", 'DOUBLE')", - ValueType.FLOAT, + ColumnType.FLOAT, ExprMacroTable.nil() ) ) @@ -398,7 +398,7 @@ public void testQuantileOnCastedString() throws Exception new ExpressionVirtualColumn( "v0", "CAST(\"dim1\", 'DOUBLE')", - ValueType.FLOAT, + ColumnType.FLOAT, ExprMacroTable.nil() ) ) @@ -479,8 +479,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java index dca89e26dbb0..99a7c4dadac5 100644 --- a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java +++ b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java @@ -25,7 +25,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -37,6 +37,7 @@ public abstract class TimestampAggregatorFactory extends AggregatorFactory { + public static final ColumnType FINALIZED_TYPE = ColumnType.ofComplex("dateTime"); final String name; @Nullable final String fieldName; @@ -199,18 +200,18 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } /** * actual type is {@link DateTime} */ @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return FINALIZED_TYPE; } @Override diff --git a/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampMinMaxAggregatorFactoryTest.java b/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampMinMaxAggregatorFactoryTest.java index 4b87f804ed35..aea3ed5266df 100644 --- a/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampMinMaxAggregatorFactoryTest.java +++ b/extensions-contrib/time-min-max/src/test/java/org/apache/druid/query/aggregation/TimestampMinMaxAggregatorFactoryTest.java @@ -29,8 +29,8 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; 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.junit.Assert; import org.junit.Test; @@ -108,13 +108,13 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("timeMax", null) .add("timeMin", null) - .add("timeMax-access", ValueType.LONG) - .add("timeMax-finalize", ValueType.COMPLEX) - .add("timeMin-access", ValueType.LONG) - .add("timeMin-finalize", ValueType.COMPLEX) + .add("timeMax-access", ColumnType.LONG) + .add("timeMax-finalize", TimestampAggregatorFactory.FINALIZED_TYPE) + .add("timeMin-access", ColumnType.LONG) + .add("timeMin-finalize", TimestampAggregatorFactory.FINALIZED_TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumn.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumn.java index 7451398baa1e..da7ce0b2c68b 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumn.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapVirtualColumn.java @@ -30,7 +30,7 @@ import org.apache.druid.query.filter.DimFilterUtils; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.virtual.VirtualColumnCacheHelper; @@ -173,7 +173,7 @@ public String getObject() @Override public ColumnCapabilities capabilities(String columnName) { - final ValueType valueType = columnName.indexOf('.') < 0 ? ValueType.COMPLEX : ValueType.STRING; + final ColumnType valueType = columnName.indexOf('.') < 0 ? ColumnType.UNKNOWN_COMPLEX : ColumnType.STRING; return new ColumnCapabilitiesImpl().setType(valueType); } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java index 18d63a90f133..c71128e1f07b 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.ObjectAggregateCombiner; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Collections; @@ -169,19 +169,10 @@ public Class classOfObject() }; } - /** - * actual type is {@link HllSketch} - */ - @Override - public ValueType getType() - { - return ValueType.COMPLEX; - } - @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return round ? ValueType.LONG : ValueType.DOUBLE; + return round ? ColumnType.LONG : ColumnType.DOUBLE; } @Nullable diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java index df68180b7bf1..ccb1f9209fa1 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -40,6 +41,7 @@ */ public class HllSketchBuildAggregatorFactory extends HllSketchAggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex(HllSketchModule.BUILD_TYPE_NAME); @JsonCreator public HllSketchBuildAggregatorFactory( @@ -54,9 +56,9 @@ public HllSketchBuildAggregatorFactory( } @Override - public String getComplexTypeName() + public ColumnType getType() { - return HllSketchModule.BUILD_TYPE_NAME; + return TYPE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java index 050cf59e1fcf..e6b6fac2a340 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchMergeAggregatorFactory.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -43,6 +44,7 @@ */ public class HllSketchMergeAggregatorFactory extends HllSketchAggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex(HllSketchModule.MERGE_TYPE_NAME); @JsonCreator public HllSketchMergeAggregatorFactory( @@ -75,9 +77,9 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre } @Override - public String getComplexTypeName() + public ColumnType getType() { - return HllSketchModule.MERGE_TYPE_NAME; + return TYPE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregator.java index 0f058d6f0d51..df4b4ec14376 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -64,9 +64,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return round ? ValueType.LONG : ValueType.DOUBLE; + return round ? ColumnType.LONG : ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java index 58722d584fa6..ff9d796edb47 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimateWithBoundsPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -70,9 +70,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java index ae60dcf312e9..a673ad2acf92 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToStringPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -87,9 +87,9 @@ public String compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchUnionPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchUnionPostAggregator.java index 4841fa42bb09..f180dbc07fb4 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchUnionPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchUnionPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -75,9 +75,9 @@ public String getName() * actual type is {@link HllSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return HllSketchMergeAggregatorFactory.TYPE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java index 3669bbdddeb0..5a4c6d11fa10 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchBaseSqlAggregator.java @@ -34,6 +34,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; @@ -118,7 +119,7 @@ public Aggregation toDruidAggregation( final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name; if (columnArg.isDirectColumnAccess() - && rowSignature.getColumnType(columnArg.getDirectColumn()).orElse(null) == ValueType.COMPLEX) { + && rowSignature.getColumnType(columnArg.getDirectColumn()).map(type -> type.is(ValueType.COMPLEX)).orElse(false)) { aggregatorFactory = new HllSketchMergeAggregatorFactory( aggregatorName, columnArg.getDirectColumn(), @@ -128,7 +129,7 @@ public Aggregation toDruidAggregation( ); } else { final RelDataType dataType = columnRexNode.getType(); - final ValueType inputType = Calcites.getValueTypeForRelDataType(dataType); + final ColumnType inputType = Calcites.getColumnTypeForRelDataType(dataType); if (inputType == null) { throw new ISE( "Cannot translate sqlTypeName[%s] to Druid type for field[%s]", diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java index 443727975fb6..2147cf051c86 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java @@ -43,7 +43,7 @@ import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.VectorColumnProcessorFactory; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -119,7 +119,7 @@ public DoublesSketchAggregatorFactory( public Aggregator factorize(final ColumnSelectorFactory metricFactory) { if (metricFactory.getColumnCapabilities(fieldName) != null - && ValueType.isNumeric(metricFactory.getColumnCapabilities(fieldName).getType())) { + && metricFactory.getColumnCapabilities(fieldName).isNumeric()) { final ColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); if (selector instanceof NilColumnValueSelector) { return new NoopDoublesSketchAggregator(); @@ -137,7 +137,7 @@ public Aggregator factorize(final ColumnSelectorFactory metricFactory) public BufferAggregator factorizeBuffered(final ColumnSelectorFactory metricFactory) { if (metricFactory.getColumnCapabilities(fieldName) != null - && ValueType.isNumeric(metricFactory.getColumnCapabilities(fieldName).getType())) { + && metricFactory.getColumnCapabilities(fieldName).isNumeric()) { final BaseDoubleColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); if (selector instanceof NilColumnValueSelector) { return new NoopDoublesSketchBufferAggregator(); @@ -350,25 +350,19 @@ public Object finalizeComputation(@Nullable final Object object) return object == null ? null : ((DoublesSketch) object).getN(); } - @Override - public String getComplexTypeName() - { - return DoublesSketchModule.DOUBLES_SKETCH; - } - /** * actual type is {@link DoublesSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return DoublesSketchModule.TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java index f158bfe0750f..cd047695f45d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchModule.java @@ -34,6 +34,7 @@ import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchRankOperatorConversion; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchSummaryOperatorConversion; import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchToHistogramOperatorConversion; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.sql.guice.SqlBindings; @@ -45,6 +46,8 @@ public class DoublesSketchModule implements DruidModule public static final String DOUBLES_SKETCH = "quantilesDoublesSketch"; public static final String DOUBLES_SKETCH_MERGE = "quantilesDoublesSketchMerge"; + public static final ColumnType TYPE = ColumnType.ofComplex(DOUBLES_SKETCH); + public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(DOUBLES_SKETCH_MERGE); public static final String DOUBLES_SKETCH_HISTOGRAM_POST_AGG = "quantilesDoublesSketchToHistogram"; public static final String DOUBLES_SKETCH_QUANTILE_POST_AGG = "quantilesDoublesSketchToQuantile"; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregator.java index 9426c6c577de..3af44132242c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -73,9 +73,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregator.java index 5cc5cb7267d7..34cdd6a58441 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregator.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Arrays; @@ -107,9 +107,9 @@ public String getName() * actual type is {@link DoublesSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilePostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilePostAggregator.java index 0845ce2a24ad..f387ee8ac547 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilePostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilePostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -61,9 +61,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregator.java index f36fe8e43bdb..a4bf3bc3ca10 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -61,9 +61,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregator.java index a1b8ebfbb22b..5ab2683e489a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -61,9 +61,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToStringPostAggregator.java index 53fca6c9899b..8857e6efbffa 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToStringPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToStringPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -58,9 +58,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.STRING; + return ColumnType.STRING; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java index 8cab843126fb..33456fadff12 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchApproxQuantileSqlAggregator.java @@ -39,8 +39,8 @@ import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchToQuantilePostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.segment.VirtualColumn; +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.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; @@ -180,7 +180,7 @@ public Aggregation toDruidAggregation( VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( plannerContext, input, - ValueType.FLOAT + ColumnType.FLOAT ); aggregatorFactory = new DoublesSketchAggregatorFactory( histogramName, diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java index d9b84e7145fd..3abb63f40f6e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchObjectSqlAggregator.java @@ -36,8 +36,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -120,7 +120,7 @@ public Aggregation toDruidAggregation( VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( plannerContext, input, - ValueType.FLOAT + ColumnType.FLOAT ); aggregatorFactory = new DoublesSketchAggregatorFactory( histogramName, diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java index 1c9cce5d27c0..708a5e434ad6 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchConstantPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Collections; import java.util.Comparator; @@ -82,9 +82,9 @@ public String getName() * actual type is {@link SketchHolder} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return SketchModule.BUILD_TYPE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java index 4f970e807872..ccda663cdf4d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchEstimatePostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; @@ -102,9 +102,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return errorBoundsStdDev != null ? ValueType.COMPLEX : ValueType.DOUBLE; + return errorBoundsStdDev != null ? SketchModule.MERGE_TYPE : ColumnType.DOUBLE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java index 87aa733a4738..31375ccc79e2 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchMergeAggregatorFactory.java @@ -24,7 +24,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; import org.apache.druid.query.aggregation.AggregatorUtil; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Collections; @@ -140,23 +140,13 @@ public Object finalizeComputation(@Nullable Object object) } } - @Override - public String getComplexTypeName() - { - if (isInputThetaSketch) { - return SketchModule.THETA_SKETCH_MERGE_AGG; - } else { - return SketchModule.THETA_SKETCH_BUILD_AGG; - } - } - /** * actual type is {@link SketchHolder} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return isInputThetaSketch ? SketchModule.MERGE_TYPE : SketchModule.BUILD_TYPE; } /** @@ -166,12 +156,12 @@ public ValueType getType() * if {@link #shouldFinalize} is NOT set, type is {@link SketchHolder} */ @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { if (shouldFinalize && errorBoundsStdDev == null) { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } - return ValueType.COMPLEX; + return getType(); } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java index 0ca00909bced..b3b785c0e064 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java @@ -32,6 +32,7 @@ import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSetIntersectOperatorConversion; import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSetNotOperatorConversion; import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchSetUnionOperatorConversion; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.sql.guice.SqlBindings; @@ -44,6 +45,8 @@ public class SketchModule implements DruidModule public static final String THETA_SKETCH_MERGE_AGG = "thetaSketchMerge"; public static final String THETA_SKETCH_BUILD_AGG = "thetaSketchBuild"; + public static final ColumnType BUILD_TYPE = ColumnType.ofComplex(THETA_SKETCH_BUILD_AGG); + public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(THETA_SKETCH_MERGE_AGG); public static final String THETA_SKETCH_ESTIMATE_POST_AGG = "thetaSketchEstimate"; public static final String THETA_SKETCH_SET_OP_POST_AGG = "thetaSketchSetOp"; diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java index 84a7533822b4..92ff6aad2909 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchSetPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.LinkedHashSet; @@ -99,9 +99,9 @@ public String getName() * actual type is {@link SketchHolder} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return SketchModule.MERGE_TYPE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java index 1f7bab9388d2..9dab4457087c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchToStringPostAggregator.java @@ -25,7 +25,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -80,9 +80,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java index b71a8cc90cab..3bdfd36a5e7b 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchBaseSqlAggregator.java @@ -33,6 +33,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; @@ -97,7 +98,7 @@ public Aggregation toDruidAggregation( final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name; if (columnArg.isDirectColumnAccess() - && rowSignature.getColumnType(columnArg.getDirectColumn()).orElse(null) == ValueType.COMPLEX) { + && rowSignature.getColumnType(columnArg.getDirectColumn()).map(type -> type.is(ValueType.COMPLEX)).orElse(false)) { aggregatorFactory = new SketchMergeAggregatorFactory( aggregatorName, columnArg.getDirectColumn(), @@ -108,7 +109,7 @@ public Aggregation toDruidAggregation( ); } else { final RelDataType dataType = columnRexNode.getType(); - final ValueType inputType = Calcites.getValueTypeForRelDataType(dataType); + final ColumnType inputType = Calcites.getColumnTypeForRelDataType(dataType); if (inputType == null) { throw new ISE( "Cannot translate sqlTypeName[%s] to Druid type for field[%s]", diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java index 3fb748f6b8a7..9d127bb114de 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactory.java @@ -41,7 +41,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.NilColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.ArrayList; @@ -293,28 +293,19 @@ public Object finalizeComputation(@Nullable final Object object) return object == null ? null : ((ArrayOfDoublesSketch) object).getEstimate(); } - @Override - public String getComplexTypeName() - { - if (metricColumns == null) { - return ArrayOfDoublesSketchModule.ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG; - } - return ArrayOfDoublesSketchModule.ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG; - } - /** * actual type is {@link ArrayOfDoublesSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return metricColumns == null ? ArrayOfDoublesSketchModule.MERGE_TYPE : ArrayOfDoublesSketchModule.BUILD_TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java index 3fdc0f4a8875..726925a0826a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchModule.java @@ -25,6 +25,7 @@ import com.google.inject.Binder; import org.apache.datasketches.tuple.arrayofdoubles.ArrayOfDoublesSketch; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.serde.ComplexMetrics; import java.util.Collections; @@ -44,6 +45,9 @@ public class ArrayOfDoublesSketchModule implements DruidModule public static final String ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG = "arrayOfDoublesSketchMerge"; public static final String ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG = "arrayOfDoublesSketchBuild"; + public static final ColumnType BUILD_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG); + public static final ColumnType MERGE_TYPE = ColumnType.ofComplex(ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG); + @Override public void configure(final Binder binder) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregator.java index 082d55aca76e..cd1933753871 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -97,9 +97,9 @@ public byte[] getCacheKey() * actual type is {@link ArrayOfDoublesSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return ArrayOfDoublesSketchModule.MERGE_TYPE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregator.java index e497d3634bc4..86930b41356a 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregator.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -88,9 +88,9 @@ public double[] compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } private static SummaryStatistics[] getStats(final ArrayOfDoublesSketch sketch) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator.java index b5f7c085597e..d9519dfef29e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -70,9 +70,9 @@ public double[] compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregator.java index 095f6b103db7..0052e06a4f72 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregator.java @@ -25,7 +25,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -53,9 +53,9 @@ public Double compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregator.java index 59cd2aab3ca1..4054b64e281e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -69,9 +69,9 @@ public double[] compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregator.java index 70035c04a38b..b4dad0b893f0 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregator.java @@ -25,7 +25,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -53,9 +53,9 @@ public Integer compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregator.java index 8d3dda14a696..60fc04e6f0a1 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregator.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -96,9 +96,9 @@ public byte[] getCacheKey() * actual type is {@link DoublesSketch} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return ArrayOfDoublesSketchModule.MERGE_TYPE; } @JsonProperty diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregator.java index 2f7c69042e84..8ed72e826c4c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -56,9 +56,9 @@ public String compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregator.java index ad1f2e508bc7..359792b05733 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -69,9 +69,9 @@ public double[] compute(final Map combinedAggregators) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } @Override diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java index 6af99b3298f3..3b9ea782c657 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java @@ -32,6 +32,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; @@ -283,19 +284,19 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("hllBuild", null) .add("hllBuildRound", null) .add("hllMerge", null) .add("hllMergeRound", null) - .add("hllBuild-access", ValueType.COMPLEX) - .add("hllBuild-finalize", ValueType.DOUBLE) - .add("hllBuildRound-access", ValueType.COMPLEX) - .add("hllBuildRound-finalize", ValueType.LONG) - .add("hllMerge-access", ValueType.COMPLEX) - .add("hllMerge-finalize", ValueType.DOUBLE) - .add("hllMergeRound-access", ValueType.COMPLEX) - .add("hllMergeRound-finalize", ValueType.LONG) + .add("hllBuild-access", HllSketchBuildAggregatorFactory.TYPE) + .add("hllBuild-finalize", ColumnType.DOUBLE) + .add("hllBuildRound-access", HllSketchBuildAggregatorFactory.TYPE) + .add("hllBuildRound-finalize", ColumnType.LONG) + .add("hllMerge-access", HllSketchMergeAggregatorFactory.TYPE) + .add("hllMerge-finalize", ColumnType.DOUBLE) + .add("hllMergeRound-access", HllSketchMergeAggregatorFactory.TYPE) + .add("hllMergeRound-finalize", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); @@ -351,9 +352,9 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) } @Override - public String getComplexTypeName() + public ColumnType getType() { - return DUMMY_TYPE_NAME; + return new ColumnType(ValueType.COMPLEX, DUMMY_TYPE_NAME, null); } @Override diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java index eb3a4660ae98..4a8b5ec80427 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchToEstimatePostAggregatorTest.java @@ -29,8 +29,8 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -113,10 +113,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("hllMerge", null) - .add("hllEstimate", ValueType.DOUBLE) - .add("hllEstimateRound", ValueType.LONG) + .add("hllEstimate", ColumnType.DOUBLE) + .add("hllEstimateRound", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 2c71fa429ebb..1a89b67dd7ef 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -173,13 +173,13 @@ public void testApproxCountDistinctHllSketch() throws Exception new ExpressionVirtualColumn( "v0", "substring(\"dim2\", 0, 1)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "v1", "concat(substring(\"dim2\", 0, 1),'x')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -410,13 +410,13 @@ public void testHllSketchPostAggs() throws Exception new ExpressionVirtualColumn( "v0", "concat(\"dim2\",'hello')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "v1", "pow(abs((\"m1\" + 100)),2)", - ValueType.DOUBLE, + ColumnType.DOUBLE, TestExprMacroTable.INSTANCE ) ) @@ -620,8 +620,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactoryTest.java index c18212cb22b7..cf1f4f95e80e 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactoryTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -127,13 +127,13 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("doublesSketch", null) .add("doublesSketchMerge", null) - .add("doublesSketch-access", ValueType.COMPLEX) - .add("doublesSketch-finalize", ValueType.LONG) - .add("doublesSketchMerge-access", ValueType.COMPLEX) - .add("doublesSketchMerge-finalize", ValueType.LONG) + .add("doublesSketch-access", DoublesSketchModule.TYPE) + .add("doublesSketch-finalize", ColumnType.LONG) + .add("doublesSketchMerge-access", DoublesSketchModule.TYPE) + .add("doublesSketchMerge-finalize", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregatorTest.java index b02cf86e324e..c3242875779c 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToCDFPostAggregatorTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -176,7 +176,7 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() .add("sketch", null) - .add("a", ValueType.DOUBLE_ARRAY) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregatorTest.java index 48261f76f996..2bfc0e53539d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToHistogramPostAggregatorTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -212,7 +212,7 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() .add("sketch", null) - .add("a", ValueType.DOUBLE_ARRAY) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregatorTest.java index 0099ef3b7467..44648977a627 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToQuantilesPostAggregatorTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -178,7 +178,7 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() .add("sketch", null) - .add("a", ValueType.DOUBLE_ARRAY) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregatorTest.java index 86b416a8abee..a06f153fab74 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchToRankPostAggregatorTest.java @@ -30,8 +30,8 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -151,7 +151,7 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() .add("sketch", null) - .add("a", ValueType.DOUBLE) + .add("a", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index bf550312b22e..37d629710d22 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -52,7 +52,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -159,7 +159,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new ExpressionVirtualColumn( "v0", "(\"m1\" * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -317,13 +317,13 @@ public void testQuantileOnCastedString() throws Exception new ExpressionVirtualColumn( "v0", "CAST(\"dim1\", 'DOUBLE')", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "v1", "(CAST(\"dim1\", 'DOUBLE') * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -471,7 +471,7 @@ public void testQuantileOnInnerQuantileQuery() throws Exception ) .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("d0", "_d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING)) .setAggregatorSpecs( new DoublesSketchAggregatorFactory("_a0:agg", "a0", 128) ) @@ -528,7 +528,7 @@ public void testDoublesSketchPostAggs() throws Exception new ExpressionVirtualColumn( "v0", "(\"cnt\" + 123)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -780,8 +780,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index 9abec0728004..bd495ce0d669 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -29,8 +29,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -92,26 +92,26 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("oldBuild", null) - .add("oldMerge", ValueType.COMPLEX) + .add("oldMerge", SketchModule.MERGE_TYPE) .add("oldMergeFinalize", null) - .add("merge", ValueType.COMPLEX) + .add("merge", SketchModule.BUILD_TYPE) .add("mergeFinalize", null) - .add("oldBuild-access", ValueType.COMPLEX) - .add("oldBuild-finalize", ValueType.DOUBLE) - .add("oldMerge-access", ValueType.COMPLEX) - .add("oldMerge-finalize", ValueType.COMPLEX) - .add("oldMergeFinalize-access", ValueType.COMPLEX) - .add("oldMergeFinalize-finalize", ValueType.DOUBLE) - .add("merge-access", ValueType.COMPLEX) - .add("merge-finalize", ValueType.COMPLEX) - .add("mergeFinalize-access", ValueType.COMPLEX) - .add("mergeFinalize-finalize", ValueType.DOUBLE) - .add("sketchEstimate", ValueType.DOUBLE) - .add("sketchEstimateStdDev", ValueType.COMPLEX) - .add("sketchSet", ValueType.COMPLEX) - .add("sketchString", ValueType.STRING) + .add("oldBuild-access", SketchModule.BUILD_TYPE) + .add("oldBuild-finalize", ColumnType.DOUBLE) + .add("oldMerge-access", SketchModule.MERGE_TYPE) + .add("oldMerge-finalize", SketchModule.MERGE_TYPE) + .add("oldMergeFinalize-access", SketchModule.MERGE_TYPE) + .add("oldMergeFinalize-finalize", ColumnType.DOUBLE) + .add("merge-access", SketchModule.BUILD_TYPE) + .add("merge-finalize", SketchModule.BUILD_TYPE) + .add("mergeFinalize-access", SketchModule.BUILD_TYPE) + .add("mergeFinalize-finalize", ColumnType.DOUBLE) + .add("sketchEstimate", ColumnType.DOUBLE) + .add("sketchEstimateStdDev", SketchModule.MERGE_TYPE) + .add("sketchSet", SketchModule.MERGE_TYPE) + .add("sketchString", ColumnType.STRING) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index 963a69c80110..31c43f7f420d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -47,7 +47,7 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -193,13 +193,13 @@ public void testApproxCountDistinctThetaSketch() throws Exception new ExpressionVirtualColumn( "v0", "substring(\"dim2\", 0, 1)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "v1", "concat(substring(\"dim2\", 0, 1),'x')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -402,7 +402,7 @@ public void testThetaSketchPostAggs() throws Exception new ExpressionVirtualColumn( "v0", "concat(\"dim2\",'hello')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -642,8 +642,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactoryTest.java index ebb722d9ee11..9e3b5e1110e1 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchAggregatorFactoryTest.java @@ -34,8 +34,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -102,10 +102,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("arrayOfDoublesSketch", null) - .add("a", ValueType.COMPLEX) - .add("b", ValueType.DOUBLE) + .add("a", ArrayOfDoublesSketchModule.BUILD_TYPE) + .add("b", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregatorTest.java index 96397c05f155..246cf910cb00 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchSetOpPostAggregatorTest.java @@ -35,8 +35,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; @@ -197,8 +197,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.COMPLEX) + .add("count", ColumnType.LONG) + .add("a", ArrayOfDoublesSketchModule.MERGE_TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregatorTest.java index e9b7d921f166..c449fcf0d695 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchTTestPostAggregatorTest.java @@ -34,8 +34,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; @@ -176,8 +176,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE_ARRAY) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregatorTest.java index 4d998137535b..549323435cc2 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToEstimatePostAggregatorTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -137,8 +137,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregatorTest.java index ac86c9c01f99..2ccc7b196901 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToMeansPostAggregatorTest.java @@ -30,8 +30,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -116,8 +116,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE_ARRAY) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregatorTest.java index 6ba6b55e0cd1..e98d444f1d57 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToNumEntriesPostAggregatorTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -138,8 +138,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.LONG) + .add("count", ColumnType.LONG) + .add("a", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregatorTest.java index ab3536f5cb4c..c09f9b81f014 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToQuantilesSketchPostAggregatorTest.java @@ -33,8 +33,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -151,8 +151,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.COMPLEX) + .add("count", ColumnType.LONG) + .add("a", ArrayOfDoublesSketchModule.MERGE_TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregatorTest.java index 30806fa19d15..fdce882ebe64 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToStringPostAggregatorTest.java @@ -30,8 +30,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -116,8 +116,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.STRING) + .add("count", ColumnType.LONG) + .add("a", ColumnType.STRING) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregatorTest.java index 00d53cd10ca7..4f42e46103db 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/tuple/ArrayOfDoublesSketchToVariancesPostAggregatorTest.java @@ -30,8 +30,8 @@ import org.apache.druid.query.aggregation.post.ConstantPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -116,8 +116,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE_ARRAY) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE_ARRAY) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java index c596141f714a..33656241e38c 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactory.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -49,6 +49,7 @@ public class BloomFilterAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex(BloomFilterSerializersModule.BLOOM_FILTER_TYPE_NAME); private static final int DEFAULT_NUM_ENTRIES = 1500; private static final Comparator COMPARATOR = Comparator.nullsFirst((o1, o2) -> { @@ -178,25 +179,19 @@ public List requiredFields() return Collections.singletonList(field.getDimension()); } - @Override - public String getComplexTypeName() - { - return BloomFilterSerializersModule.BLOOM_FILTER_TYPE_NAME; - } - /** * actual type is {@link ByteBuffer} containing {@link BloomKFilter} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return TYPE; } @Override @@ -254,8 +249,7 @@ private BaseBloomFilterAggregator factorizeInternal(ColumnSelectorFactory column ColumnCapabilities capabilities = columnFactory.getColumnCapabilities(field.getDimension()); if (capabilities != null) { - ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case STRING: return new StringBloomFilterAggregator( columnFactory.makeDimensionSelector(field), @@ -291,7 +285,7 @@ private BaseBloomFilterAggregator factorizeInternal(ColumnSelectorFactory column throw new IAE( "Cannot create bloom filter %s for invalid column type [%s]", onHeap ? "aggregator" : "buffer aggregator", - type + capabilities.asTypeString() ); } } else { diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java index f934520830ee..2fe958922e91 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregator.java @@ -38,8 +38,8 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.segment.VirtualColumn; +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.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -145,7 +145,7 @@ public Aggregation toDruidAggregation( // No existing match found. Create a new one. - ValueType valueType = Calcites.getValueTypeForRelDataType(inputOperand.getType()); + ColumnType valueType = Calcites.getColumnTypeForRelDataType(inputOperand.getType()); final DimensionSpec spec; if (input.isDirectColumnAccess()) { spec = new DefaultDimensionSpec( diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExprMacro.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExprMacro.java index ca28cc0fecad..637daff3689c 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExprMacro.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; 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.query.filter.BloomKFilter; import javax.annotation.Nonnull; @@ -82,7 +82,7 @@ public ExprEval eval(final ObjectBinding bindings) ExprEval evaluated = arg.eval(bindings); boolean matches = false; - switch (evaluated.type()) { + switch (evaluated.type().getType()) { case STRING: String stringVal = (String) evaluated.value(); if (stringVal == null) { @@ -127,9 +127,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactoryTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactoryTest.java index dc9d0b924541..692494f5988b 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactoryTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/BloomFilterAggregatorFactoryTest.java @@ -27,8 +27,8 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -58,13 +58,13 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("bloom", ValueType.COMPLEX) - .add("bloomMerge", ValueType.COMPLEX) - .add("bloom-access", ValueType.COMPLEX) - .add("bloom-finalize", ValueType.COMPLEX) - .add("bloomMerge-access", ValueType.COMPLEX) - .add("bloomMerge-finalize", ValueType.COMPLEX) + .add("count", ColumnType.LONG) + .add("bloom", BloomFilterAggregatorFactory.TYPE) + .add("bloomMerge", BloomFilterAggregatorFactory.TYPE) + .add("bloom-access", BloomFilterAggregatorFactory.TYPE) + .add("bloom-finalize", BloomFilterAggregatorFactory.TYPE) + .add("bloomMerge-access", BloomFilterAggregatorFactory.TYPE) + .add("bloomMerge-finalize", BloomFilterAggregatorFactory.TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java index effb87d23ed2..a60a7b068f2a 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java @@ -49,7 +49,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -315,7 +315,7 @@ public void testBloomFilterAggLong() throws Exception ImmutableList.of( new BloomFilterAggregatorFactory( "a0:agg", - new DefaultDimensionSpec("l1", "a0:l1", ValueType.LONG), + new DefaultDimensionSpec("l1", "a0:l1", ColumnType.LONG), TEST_NUM_ENTRIES ) ) @@ -360,7 +360,7 @@ public void testBloomFilterAggLongVirtualColumn() throws Exception new ExpressionVirtualColumn( "v0", "(\"l1\" * 2)", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) @@ -414,7 +414,7 @@ public void testBloomFilterAggFloatVirtualColumn() throws Exception new ExpressionVirtualColumn( "v0", "(\"f1\" * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -468,7 +468,7 @@ public void testBloomFilterAggDoubleVirtualColumn() throws Exception new ExpressionVirtualColumn( "v0", "(\"d1\" * 2)", - ValueType.DOUBLE, + ColumnType.DOUBLE, TestExprMacroTable.INSTANCE ) ) @@ -519,7 +519,7 @@ public void testEmptyTimeseriesResults() throws Exception ), new BloomFilterAggregatorFactory( "a1:agg", - new DefaultDimensionSpec("l1", "a1:l1", ValueType.LONG), + new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG), TEST_NUM_ENTRIES ) ) @@ -557,8 +557,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( @@ -572,7 +572,7 @@ public void testGroupByAggregatorDefaultValues() throws Exception new FilteredAggregatorFactory( new BloomFilterAggregatorFactory( "a1:agg", - new DefaultDimensionSpec("l1", "a1:l1", ValueType.LONG), + new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG), TEST_NUM_ENTRIES ), selector("dim1", "nonexistent", null) diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java index 24e3b6b97267..97120c22c881 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java @@ -37,7 +37,7 @@ import org.apache.druid.query.filter.BloomKFilterHolder; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.OrDimFilter; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.DruidOperatorTable; @@ -157,7 +157,7 @@ public void testBloomFilterVirtualColumn() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim1\",'-foo')", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim1\",'-foo')", ColumnType.STRING)) .filters( new BloomDimFilter("v0", BloomKFilterHolder.fromBloomKFilter(filter), null) ) @@ -188,7 +188,7 @@ public void testBloomFilterVirtualColumnNumber() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "(2 * CAST(\"dim1\", 'DOUBLE'))", ValueType.FLOAT) + expressionVirtualColumn("v0", "(2 * CAST(\"dim1\", 'DOUBLE'))", ColumnType.FLOAT) ) .filters( new BloomDimFilter("v0", BloomKFilterHolder.fromBloomKFilter(filter), null) diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java index cde33fd605f2..3524b2f32240 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorFactory.java @@ -31,6 +31,7 @@ import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.ObjectAggregateCombiner; import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; @@ -38,7 +39,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -51,6 +52,7 @@ @JsonTypeName("approxHistogram") public class ApproximateHistogramAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("approximateHistogram"); protected final String name; protected final String fieldName; @@ -122,7 +124,7 @@ public boolean canVectorize(ColumnInspector columnInspector) /* skip vectorization for string types which may be parseable to numbers. There is no vector equivalent of string value selector*/ ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); - return (capabilities != null) && capabilities.getType().isNumeric(); + return capabilities != null && capabilities.isNumeric(); } @Override @@ -321,28 +323,22 @@ public byte[] getCacheKey() return builder.build(); } - @Override - public String getComplexTypeName() - { - return "approximateHistogram"; - } - /** * actual type is {@link ApproximateHistogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } /** * actual type is {@link ApproximateHistogram} if {@link #finalizeAsBase64Binary} is set, else {@link Histogram} */ @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return finalizeAsBase64Binary ? TYPE : HistogramAggregatorFactory.TYPE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java index 7a8a80cc21b2..9d1d4e1423fe 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingAggregatorFactory.java @@ -111,7 +111,7 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory metricVector public boolean canVectorize(ColumnInspector columnInspector) { ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); - return (capabilities != null) && (capabilities.getType() == ValueType.COMPLEX); + return capabilities != null && capabilities.is(ValueType.COMPLEX); } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/BucketsPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/BucketsPostAggregator.java index 839645f9b59c..d42c15f22158 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/BucketsPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/BucketsPostAggregator.java @@ -25,10 +25,11 @@ import com.google.common.collect.Sets; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Map; import java.util.Objects; @@ -73,9 +74,9 @@ public Object compute(Map values) * actual type is {@link Histogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return HistogramAggregatorFactory.TYPE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java index 8d23316ac766..01e22c5f9b49 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/CustomBucketsPostAggregator.java @@ -24,10 +24,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.collect.Sets; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Map; @@ -66,9 +67,9 @@ public Object compute(Map values) * actual type is {@link Histogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return HistogramAggregatorFactory.TYPE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java index a64b87950ea2..cad63c79a220 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/EqualBucketsPostAggregator.java @@ -25,10 +25,11 @@ import com.google.common.collect.Sets; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Map; import java.util.Objects; @@ -70,9 +71,9 @@ public Object compute(Map values) * actual type is {@link Histogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return HistogramAggregatorFactory.TYPE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregator.java index eed1c9e7d380..5c7d2452d252 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregator.java @@ -22,6 +22,7 @@ import com.google.common.primitives.Longs; import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -29,6 +30,7 @@ public class FixedBucketsHistogramAggregator implements Aggregator { public static final String TYPE_NAME = "fixedBucketsHistogram"; + public static final ColumnType TYPE = ColumnType.ofComplex(TYPE_NAME); public static final Comparator COMPARATOR = new Comparator() { diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java index 705c420774b8..831a111b6791 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -112,8 +112,7 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelect if (null == capabilities) { throw new IAE("could not find the column type for column %s", fieldName); } - ValueType type = capabilities.getType(); - if (type.isNumeric()) { + if (capabilities.isNumeric()) { return new FixedBucketsHistogramVectorAggregator( columnSelectorFactory.makeValueSelector(fieldName), lowerLimit, @@ -122,7 +121,7 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelect outlierHandlingMode ); } else { - throw new IAE("cannot vectorize fixed bucket histogram aggregation for type %s", type); + throw new IAE("cannot vectorize fixed bucket histogram aggregation for type %s", capabilities.asTypeString()); } } @@ -130,7 +129,7 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelect public boolean canVectorize(ColumnInspector columnInspector) { ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); - return (capabilities != null) && capabilities.getType().isNumeric(); + return capabilities != null && capabilities.isNumeric(); } @Override @@ -279,28 +278,22 @@ public List requiredFields() return Collections.singletonList(fieldName); } - @Override - public String getComplexTypeName() - { - return FixedBucketsHistogramAggregator.TYPE_NAME; - } - /** * actual type is {@link FixedBucketsHistogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return FixedBucketsHistogramAggregator.TYPE; } /** * actual type is {@link FixedBucketsHistogram} if {@link #finalizeAsBase64Binary} is set */ @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return finalizeAsBase64Binary ? ValueType.COMPLEX : ValueType.STRING; + return finalizeAsBase64Binary ? FixedBucketsHistogramAggregator.TYPE : ColumnType.STRING; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MaxPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MaxPostAggregator.java index 6eb039501834..7f0f5ba02b79 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MaxPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MaxPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -74,9 +74,9 @@ public Object compute(Map values) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MinPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MinPostAggregator.java index a238a5e89a83..5d856f707bd4 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MinPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/MinPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -76,9 +76,9 @@ public Object compute(Map values) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilePostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilePostAggregator.java index fdda08f07a73..f2efe05550d6 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilePostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilePostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.Map; @@ -85,9 +85,9 @@ public Object compute(Map values) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilesPostAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilesPostAggregator.java index 07edf04b4034..e2cca39727ea 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilesPostAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/QuantilesPostAggregator.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Arrays; import java.util.Comparator; @@ -97,9 +97,10 @@ public Object compute(Map values) * actual type is {@link Quantiles} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + // todo: ??? + return ColumnType.UNKNOWN_COMPLEX; } @Override diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java index eb3ae4335f97..bb4949e9c5b3 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregator.java @@ -38,8 +38,8 @@ import org.apache.druid.query.aggregation.histogram.FixedBucketsHistogramAggregatorFactory; import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator; import org.apache.druid.segment.VirtualColumn; +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.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; @@ -233,7 +233,7 @@ public Aggregation toDruidAggregation( VirtualColumn virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( plannerContext, input, - ValueType.FLOAT + ColumnType.FLOAT ); aggregatorFactory = new FixedBucketsHistogramAggregatorFactory( histogramName, diff --git a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java index 529fa108d1b9..21af3c500934 100644 --- a/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java +++ b/extensions-core/histogram/src/main/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregator.java @@ -39,6 +39,7 @@ import org.apache.druid.query.aggregation.histogram.ApproximateHistogramFoldingAggregatorFactory; import org.apache.druid.query.aggregation.histogram.QuantilePostAggregator; import org.apache.druid.segment.VirtualColumn; +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.virtual.ExpressionVirtualColumn; @@ -169,7 +170,7 @@ public Aggregation toDruidAggregation( // No existing match found. Create a new one. if (input.isDirectColumnAccess()) { - if (rowSignature.getColumnType(input.getDirectColumn()).orElse(null) == ValueType.COMPLEX) { + if (rowSignature.getColumnType(input.getDirectColumn()).map(type -> type.is(ValueType.COMPLEX)).orElse(false)) { aggregatorFactory = new ApproximateHistogramFoldingAggregatorFactory( histogramName, input.getDirectColumn(), @@ -192,7 +193,7 @@ public Aggregation toDruidAggregation( } } else { final VirtualColumn virtualColumn = - virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, input, ValueType.FLOAT); + virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, input, ColumnType.FLOAT); aggregatorFactory = new ApproximateHistogramAggregatorFactory( histogramName, virtualColumn.getOutputName(), diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java index 5c8bfd1d9fcd..572b2e6c772a 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramAggregatorTest.java @@ -23,13 +23,13 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.TestFloatColumnSelector; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -145,12 +145,12 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("approxHisto", ValueType.COMPLEX) - .add("approxHistoBin", ValueType.COMPLEX) - .add("approxHisto-access", ValueType.COMPLEX) - .add("approxHisto-finalize", ValueType.COMPLEX) - .add("approxHistoBin-access", ValueType.COMPLEX) - .add("approxHistoBin-finalize", ValueType.COMPLEX) + .add("approxHisto", null) + .add("approxHistoBin", ApproximateHistogramAggregatorFactory.TYPE) + .add("approxHisto-access", ApproximateHistogramAggregatorFactory.TYPE) + .add("approxHisto-finalize", HistogramAggregatorFactory.TYPE) + .add("approxHistoBin-access", ApproximateHistogramAggregatorFactory.TYPE) + .add("approxHistoBin-finalize", ApproximateHistogramAggregatorFactory.TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingVectorAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingVectorAggregatorTest.java index ee7283b20a82..3f813a5a1960 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingVectorAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramFoldingVectorAggregatorTest.java @@ -21,7 +21,7 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorObjectSelector; import org.easymock.EasyMock; @@ -64,13 +64,13 @@ public void setup() expect(vectorColumnSelectorFactory.makeObjectSelector("field")) .andReturn(vectorObjectSelector).anyTimes(); expect(vectorColumnSelectorFactory.getColumnCapabilities("field")).andReturn( - new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX) + new ColumnCapabilitiesImpl().setType(ApproximateHistogramAggregatorFactory.TYPE) ); expect(vectorColumnSelectorFactory.getColumnCapabilities("string_field")).andReturn( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) ); expect(vectorColumnSelectorFactory.getColumnCapabilities("double_field")).andReturn( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) ); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java index d6af24b929e8..bcdc7d46e98c 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramPostAggregatorTest.java @@ -21,11 +21,12 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; +import org.apache.druid.query.aggregation.HistogramAggregatorFactory; import org.apache.druid.query.aggregation.TestFloatColumnSelector; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -93,14 +94,14 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("approxHisto", ValueType.COMPLEX) - .add("bucket", ValueType.COMPLEX) - .add("equal", ValueType.COMPLEX) - .add("custom", ValueType.COMPLEX) - .add("min", ValueType.DOUBLE) - .add("max", ValueType.DOUBLE) - .add("quantile", ValueType.FLOAT) - .add("quantiles", ValueType.COMPLEX) + .add("approxHisto", null) + .add("bucket", HistogramAggregatorFactory.TYPE) + .add("equal", HistogramAggregatorFactory.TYPE) + .add("custom", HistogramAggregatorFactory.TYPE) + .add("min", ColumnType.DOUBLE) + .add("max", ColumnType.DOUBLE) + .add("quantile", ColumnType.FLOAT) + .add("quantiles", ColumnType.UNKNOWN_COMPLEX) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregatorTest.java index 9958194a74e6..da4d079c06c3 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramVectorAggregatorTest.java @@ -23,7 +23,7 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -59,7 +59,7 @@ public void setup() EasyMock.replay(vectorValueSelector_2); ColumnCapabilities columnCapabilities - = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE); + = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE); vectorColumnSelectorFactory = createMock(VectorColumnSelectorFactory.class); expect(vectorColumnSelectorFactory.getColumnCapabilities("field_1")).andReturn(columnCapabilities).anyTimes(); expect(vectorColumnSelectorFactory.makeValueSelector("field_1")) @@ -68,10 +68,10 @@ public void setup() expect(vectorColumnSelectorFactory.makeValueSelector("field_2")) .andReturn(vectorValueSelector_2).anyTimes(); expect(vectorColumnSelectorFactory.getColumnCapabilities("string_field")).andReturn( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) ); expect(vectorColumnSelectorFactory.getColumnCapabilities("complex_field")).andReturn( - new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX) + new ColumnCapabilitiesImpl().setType(ApproximateHistogramAggregatorFactory.TYPE) ); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramBufferAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramBufferAggregatorTest.java index dece34597b74..36a78891c0e7 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramBufferAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramBufferAggregatorTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -167,11 +167,11 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() .add("fixedHisto", null) - .add("fixedHistoBin", ValueType.COMPLEX) - .add("fixedHisto-access", ValueType.COMPLEX) - .add("fixedHisto-finalize", ValueType.STRING) - .add("fixedHistoBin-access", ValueType.COMPLEX) - .add("fixedHistoBin-finalize", ValueType.COMPLEX) + .add("fixedHistoBin", FixedBucketsHistogramAggregator.TYPE) + .add("fixedHisto-access", FixedBucketsHistogramAggregator.TYPE) + .add("fixedHisto-finalize", ColumnType.STRING) + .add("fixedHistoBin-access", FixedBucketsHistogramAggregator.TYPE) + .add("fixedHistoBin-finalize", FixedBucketsHistogramAggregator.TYPE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramVectorAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramVectorAggregatorTest.java index 78f74edf7483..93c9d690c283 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramVectorAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramVectorAggregatorTest.java @@ -23,7 +23,7 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -58,7 +58,7 @@ public void setup() EasyMock.replay(vectorValueSelector_2); ColumnCapabilities columnCapabilities - = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE); + = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE); vectorColumnSelectorFactory = createMock(VectorColumnSelectorFactory.class); expect(vectorColumnSelectorFactory.getColumnCapabilities("field_1")).andReturn(columnCapabilities).anyTimes(); expect(vectorColumnSelectorFactory.makeValueSelector("field_1")) diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java index 35341c24de71..4c03d2bac883 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -45,7 +45,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -158,7 +158,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new ExpressionVirtualColumn( "v0", "(\"m1\" * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -256,13 +256,13 @@ public void testQuantileOnCastedString() throws Exception new ExpressionVirtualColumn( "v0", "CAST(\"dim1\", 'DOUBLE')", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "v1", "(CAST(\"dim1\", 'DOUBLE') * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -575,8 +575,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 720d2fa8b1ca..4f03df0ae944 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -45,7 +45,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -142,7 +142,7 @@ public void testQuantileOnFloatAndLongs() throws Exception new ExpressionVirtualColumn( "v0", "(\"m1\" * 2)", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -339,7 +339,7 @@ public void testQuantileOnCastedString() throws Exception new ExpressionVirtualColumn( "v0", "CAST(\"dim1\", 'DOUBLE')", - ValueType.FLOAT, + ColumnType.FLOAT, ExprMacroTable.nil() ) ) @@ -414,8 +414,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/PvaluefromZscorePostAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/PvaluefromZscorePostAggregator.java index a835593eaf14..c25df5fa15f4 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/PvaluefromZscorePostAggregator.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/PvaluefromZscorePostAggregator.java @@ -31,7 +31,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Collections; import java.util.Comparator; @@ -109,9 +109,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/ZtestPostAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/ZtestPostAggregator.java index 6e8e03324c48..2a647bb072a7 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/ZtestPostAggregator.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/teststats/ZtestPostAggregator.java @@ -30,7 +30,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Collections; import java.util.Comparator; @@ -123,9 +123,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/StandardDeviationPostAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/StandardDeviationPostAggregator.java index c39ac8b66b7d..f7d2bc60268c 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/StandardDeviationPostAggregator.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/StandardDeviationPostAggregator.java @@ -30,7 +30,7 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -90,9 +90,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java index 2894c0195884..2f69100594a9 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactory.java @@ -43,6 +43,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -60,6 +61,8 @@ public class VarianceAggregatorFactory extends AggregatorFactory { private static final String VARIANCE_TYPE_NAME = "variance"; + public static final ColumnType TYPE = ColumnType.ofComplex(VARIANCE_TYPE_NAME); + protected final String fieldName; protected final String name; @Nullable @@ -93,25 +96,19 @@ public VarianceAggregatorFactory( this(name, fieldName, null, null); } - @Override - public String getComplexTypeName() - { - return VARIANCE_TYPE_NAME; - } - /** * actual type is {@link VarianceAggregatorCollector} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override @@ -373,6 +370,7 @@ public int hashCode() private String getTypeString(ColumnInspector columnInspector) { + // todo: make this better... why strings? String type = inputType; if (type == null) { ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); diff --git a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java index 617067aee6d8..51c7b02e41bf 100644 --- a/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java +++ b/extensions-core/stats/src/main/java/org/apache/druid/query/aggregation/variance/sql/BaseVarianceSqlAggregator.java @@ -36,8 +36,8 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -82,7 +82,7 @@ public Aggregation toDruidAggregation( final AggregatorFactory aggregatorFactory; final RelDataType dataType = inputOperand.getType(); - final ValueType inputType = Calcites.getValueTypeForRelDataType(dataType); + final ColumnType inputType = Calcites.getColumnTypeForRelDataType(dataType); final DimensionSpec dimensionSpec; final String aggName = StringUtils.format("%s:agg", name); final SqlAggFunction func = calciteFunction(); @@ -98,14 +98,14 @@ public Aggregation toDruidAggregation( dimensionSpec = new DefaultDimensionSpec(virtualColumn.getOutputName(), null, inputType); } - switch (inputType) { - case LONG: - case DOUBLE: - case FLOAT: - inputTypeName = StringUtils.toLowerCase(inputType.name()); - break; - default: - throw new IAE("VarianceSqlAggregator[%s] has invalid inputType[%s]", func, inputType); + if (inputType == null) { + throw new IAE("VarianceSqlAggregator[%s] has invalid inputType", func); + } + + if (inputType.isNumeric()) { + inputTypeName = StringUtils.toLowerCase(inputType.getType().name()); + } else { + throw new IAE("VarianceSqlAggregator[%s] has invalid inputType[%s]", func, inputType.asTypeString()); } @@ -122,7 +122,7 @@ public Aggregation toDruidAggregation( inputTypeName ); - if (func == SqlStdOperatorTable.STDDEV_POP + if (func == SqlStdOperatorTable.STDDEV_POP || func == SqlStdOperatorTable.STDDEV_SAMP || func == SqlStdOperatorTable.STDDEV) { postAggregator = new StandardDeviationPostAggregator( diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryTest.java index 7f0bbaa1da55..d3fded4dfb8b 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryTest.java @@ -27,8 +27,8 @@ import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -59,13 +59,13 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("variance", null) .add("varianceFold", null) - .add("variance-access", ValueType.COMPLEX) - .add("variance-finalize", ValueType.DOUBLE) - .add("varianceFold-access", ValueType.COMPLEX) - .add("varianceFold-finalize", ValueType.DOUBLE) + .add("variance-access", VarianceAggregatorFactory.TYPE) + .add("variance-finalize", ColumnType.DOUBLE) + .add("varianceFold-access", VarianceAggregatorFactory.TYPE) + .add("varianceFold-finalize", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryUnitTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryUnitTest.java index 25a513002162..4b1d39e49504 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryUnitTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorFactoryUnitTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -101,7 +101,7 @@ public void factorizeVectorForVarianceShouldReturnObjectVectorAggregator() @Test public void factorizeVectorForComplexShouldReturnObjectVectorAggregator() { - mockType(ValueType.COMPLEX); + mockType(VarianceAggregatorFactory.TYPE); VectorAggregator agg = target.factorizeVector(selectorFactory); Assert.assertNotNull(agg); Assert.assertEquals(VarianceObjectVectorAggregator.class, agg.getClass()); @@ -110,7 +110,7 @@ public void factorizeVectorForComplexShouldReturnObjectVectorAggregator() @Test public void factorizeBufferedForComplexShouldReturnObjectVectorAggregator() { - mockType(ValueType.COMPLEX); + mockType(VarianceAggregatorFactory.TYPE); BufferAggregator agg = target.factorizeBuffered(metricFactory); Assert.assertNotNull(agg); Assert.assertEquals(VarianceBufferAggregator.ObjectVarianceAggregator.class, agg.getClass()); @@ -119,7 +119,7 @@ public void factorizeBufferedForComplexShouldReturnObjectVectorAggregator() @Test public void factorizeForComplexShouldReturnObjectVectorAggregator() { - mockType(ValueType.COMPLEX); + mockType(VarianceAggregatorFactory.TYPE); Aggregator agg = target.factorize(metricFactory); Assert.assertNotNull(agg); Assert.assertEquals(VarianceAggregator.ObjectVarianceAggregator.class, agg.getClass()); @@ -147,10 +147,10 @@ public void equalsContract() .verify(); } - private void mockType(ValueType type) + private void mockType(ColumnType type) { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); Mockito.doReturn(capabilities).when(metricFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(type).when(capabilities).getType(); + Mockito.doReturn(type.getType()).when(capabilities).getType(); } } diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorTest.java index c6c8989dc0a8..d2be00b79d2d 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceAggregatorTest.java @@ -25,7 +25,6 @@ import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; @@ -135,7 +134,7 @@ public void testObjectVarianceBufferAggregatorWithZeroCount() TestObjectColumnSelector selector = new TestObjectColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX)); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(new ColumnCapabilitiesImpl().setType(VarianceAggregatorFactory.TYPE)); EasyMock.replay(colSelectorFactory); VarianceBufferAggregator agg = (VarianceBufferAggregator) aggFactory.factorizeBuffered( diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index e67f09f2399e..6253e643cfaf 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -45,7 +45,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -390,9 +390,9 @@ public void testStdDevWithVirtualColumns() throws Exception .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) .virtualColumns( - BaseCalciteQueryTest.expressionVirtualColumn("v0", "(\"d1\" * 7)", ValueType.DOUBLE), - BaseCalciteQueryTest.expressionVirtualColumn("v1", "(\"f1\" * 7)", ValueType.FLOAT), - BaseCalciteQueryTest.expressionVirtualColumn("v2", "(\"l1\" * 7)", ValueType.LONG) + BaseCalciteQueryTest.expressionVirtualColumn("v0", "(\"d1\" * 7)", ColumnType.DOUBLE), + BaseCalciteQueryTest.expressionVirtualColumn("v1", "(\"f1\" * 7)", ColumnType.FLOAT), + BaseCalciteQueryTest.expressionVirtualColumn("v2", "(\"l1\" * 7)", ColumnType.LONG) ) .aggregators( ImmutableList.of( @@ -474,7 +474,7 @@ public void testVariancesOnCastedString() throws Exception .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - new ExpressionVirtualColumn("v0", "CAST(\"dim1\", 'DOUBLE')", ValueType.DOUBLE, ExprMacroTable.nil()) + new ExpressionVirtualColumn("v0", "CAST(\"dim1\", 'DOUBLE')", ColumnType.DOUBLE, ExprMacroTable.nil()) ) .granularity(Granularities.ALL) .aggregators( @@ -570,8 +570,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( diff --git a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java index 924f495efbd0..aedb7edabd74 100644 --- a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java +++ b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java @@ -24,7 +24,7 @@ import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr; import org.apache.druid.math.expr.ExprMacroTable.ExprMacro; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.expression.ExprUtils; import java.util.List; @@ -101,7 +101,7 @@ public boolean canVectorize(InputBindingInspector inspector) } @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { return null; } diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java index 8af1e93f1ac3..e8381572c663 100644 --- a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java +++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java @@ -32,7 +32,7 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery.ResultFormat; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -79,7 +79,7 @@ public void testSleepFunction() throws Exception new ExpressionVirtualColumn( "v0", "sleep(\"m1\")", - ValueType.STRING, + ColumnType.STRING, createMacroTable() ) ) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java index b3d861061951..846826a4c02d 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java @@ -38,6 +38,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -106,7 +107,7 @@ public static Map getTypeHelperMap(DimensionsSpec Map typeHelperMap = new HashMap<>(); for (DimensionSchema dimensionSchema : dimensionsSpec.getDimensions()) { IndexSerdeTypeHelper typeHelper; - switch (dimensionSchema.getValueType()) { + switch (dimensionSchema.getColumnType().getType()) { case STRING: typeHelper = STRING_HELPER; break; @@ -120,7 +121,7 @@ public static Map getTypeHelperMap(DimensionsSpec typeHelper = DOUBLE_HELPER; break; default: - throw new IAE("Invalid type: [%s]", dimensionSchema.getValueType()); + throw new IAE("Invalid type: [%s]", dimensionSchema.getColumnType()); } typeHelperMap.put(dimensionSchema.getName(), typeHelper); } @@ -340,24 +341,24 @@ public static SerializeResult toBytes( parseExceptionMessages.add(e.getMessage()); } - final ValueType type = aggFactory.getType(); + final ColumnType type = aggFactory.getType(); if (agg.isNull()) { out.writeByte(NullHandling.IS_NULL_BYTE); } else { out.writeByte(NullHandling.IS_NOT_NULL_BYTE); - if (ValueType.FLOAT.equals(type)) { + if (type.is(ValueType.FLOAT)) { out.writeFloat(agg.getFloat()); - } else if (ValueType.LONG.equals(type)) { + } else if (type.is(ValueType.LONG)) { WritableUtils.writeVLong(out, agg.getLong()); - } else if (ValueType.DOUBLE.equals(type)) { + } else if (type.is(ValueType.DOUBLE)) { out.writeDouble(agg.getDouble()); - } else if (ValueType.COMPLEX.equals(type)) { + } else if (type.is(ValueType.COMPLEX)) { Object val = agg.get(); - ComplexMetricSerde serde = getComplexMetricSerde(aggFactory.getComplexTypeName()); + ComplexMetricSerde serde = getComplexMetricSerde(type.getComplexTypeName()); writeBytes(serde.toBytes(val), out); } else { - throw new IAE("Unable to serialize type[%s]", type); + throw new IAE("Unable to serialize type[%s]", type.asTypeString()); } } } @@ -471,21 +472,21 @@ public static InputRow fromBytes( for (int i = 0; i < metricSize; i++) { final String metric = readString(in); final AggregatorFactory agg = getAggregator(metric, aggs, i); - final ValueType type = agg.getType(); + final ColumnType type = agg.getType(); final byte metricNullability = in.readByte(); if (metricNullability == NullHandling.IS_NULL_BYTE) { // metric value is null. continue; } - if (ValueType.FLOAT.equals(type)) { + if (type.is(ValueType.FLOAT)) { event.put(metric, in.readFloat()); - } else if (ValueType.LONG.equals(type)) { + } else if (type.is(ValueType.LONG)) { event.put(metric, WritableUtils.readVLong(in)); - } else if (ValueType.DOUBLE.equals(type)) { + } else if (type.is(ValueType.DOUBLE)) { event.put(metric, in.readDouble()); } else { - ComplexMetricSerde serde = getComplexMetricSerde(agg.getComplexTypeName()); + ComplexMetricSerde serde = getComplexMetricSerde(agg.getType().getComplexTypeName()); byte[] value = readBytes(in); event.put(metric, serde.fromBytes(value, 0, value.length)); } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java index fddcbcf3e183..3bb8ff886831 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java @@ -37,7 +37,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; @@ -104,13 +104,13 @@ public void testSerde() final AggregatorFactory mockedAggregatorFactory = EasyMock.createMock(AggregatorFactory.class); EasyMock.expect(mockedAggregatorFactory.factorize(EasyMock.anyObject(ColumnSelectorFactory.class))).andReturn(mockedAggregator); - EasyMock.expect(mockedAggregatorFactory.getType()).andReturn(ValueType.DOUBLE).anyTimes(); + EasyMock.expect(mockedAggregatorFactory.getType()).andReturn(ColumnType.DOUBLE).anyTimes(); EasyMock.expect(mockedAggregatorFactory.getName()).andReturn("mockedAggregator").anyTimes(); final AggregatorFactory mockedNullAggregatorFactory = EasyMock.createMock(AggregatorFactory.class); EasyMock.expect(mockedNullAggregatorFactory.factorize(EasyMock.anyObject(ColumnSelectorFactory.class))).andReturn(mockedNullAggregator); EasyMock.expect(mockedNullAggregatorFactory.getName()).andReturn("mockedNullAggregator").anyTimes(); - EasyMock.expect(mockedNullAggregatorFactory.getType()).andReturn(ValueType.DOUBLE).anyTimes(); + EasyMock.expect(mockedNullAggregatorFactory.getType()).andReturn(ColumnType.DOUBLE).anyTimes(); EasyMock.replay(mockedAggregatorFactory, mockedNullAggregatorFactory); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 9f2af4211b0e..cdb637f9564c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -80,6 +80,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -870,7 +871,7 @@ private static DimensionsSpec createDimensionsSpec(List> loadSegments( } private static DimensionSchema createDimensionSchema( - ValueType type, + TypeSignature type, String name, MultiValueHandling multiValueHandling, boolean hasBitmapIndexes ) { - switch (type) { + switch (type.getType()) { case FLOAT: Preconditions.checkArgument( multiValueHandling == null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index 8e3bfe7108a2..f48d65b4c70f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -54,7 +54,7 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter; @@ -224,9 +224,9 @@ private static class IntermediateRowColumnProcessorFactory implements ColumnProc private static final IntermediateRowColumnProcessorFactory INSTANCE = new IntermediateRowColumnProcessorFactory(); @Override - public ValueType defaultType() + public ColumnType defaultType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 9adb4c231f02..43d503d3170a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -101,6 +101,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.SpatialIndex; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.CompressionFactory.LongEncodingStrategy; @@ -1929,7 +1930,7 @@ Map getQueryableIndexMap() private static ColumnHolder createColumn(DimensionSchema dimensionSchema) { - return new TestColumn(dimensionSchema.getValueType()); + return new TestColumn(dimensionSchema.getColumnType()); } private static ColumnHolder createColumn(AggregatorFactory aggregatorFactory) @@ -1941,12 +1942,12 @@ private static class TestColumn implements ColumnHolder { private final ColumnCapabilities columnCapabilities; - TestColumn(ValueType type) + TestColumn(ColumnType type) { columnCapabilities = new ColumnCapabilitiesImpl() .setType(type) - .setDictionaryEncoded(type == ValueType.STRING) // set a fake value to make string columns - .setHasBitmapIndexes(type == ValueType.STRING) + .setDictionaryEncoded(type.is(ValueType.STRING)) // set a fake value to make string columns + .setHasBitmapIndexes(type.is(ValueType.STRING)) .setHasSpatialIndexes(false) .setHasMultipleValues(false); } diff --git a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java index d263b08122e9..9d6aa580f9e1 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DruidMetrics.java @@ -49,7 +49,7 @@ public static int findNumComplexAggs(List aggs) { int retVal = 0; for (AggregatorFactory agg : aggs) { - if (ValueType.COMPLEX.equals(agg.getType())) { + if (agg.getType().is(ValueType.COMPLEX)) { retVal++; } } diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index c404c5537b92..4206732cfc69 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -28,8 +28,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import javax.annotation.Nullable; import java.util.Arrays; @@ -69,7 +69,7 @@ private InlineDataSource( @JsonCreator private static InlineDataSource fromJson( @JsonProperty("columnNames") List columnNames, - @JsonProperty("columnTypes") List columnTypes, + @JsonProperty("columnTypes") List columnTypes, @JsonProperty("rows") List rows ) { @@ -83,7 +83,7 @@ private static InlineDataSource fromJson( for (int i = 0; i < columnNames.size(); i++) { final String name = columnNames.get(i); - final ValueType type = columnTypes != null ? columnTypes.get(i) : null; + final ColumnType type = columnTypes != null ? columnTypes.get(i) : null; builder.add(name, type); } @@ -120,7 +120,7 @@ public List getColumnNames() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public List getColumnTypes() + public List getColumnTypes() { if (IntStream.range(0, signature.size()).noneMatch(i -> signature.getColumnType(i).isPresent())) { // All types are null; return null for columnTypes so it doesn't show up in serialized JSON. diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 2acbe66dc325..ec28db70a8c5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -21,12 +21,12 @@ import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.Cacheable; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -217,34 +217,18 @@ public AggregatorFactory getMergingFactory(AggregatorFactory other) throws Aggre * {@link #deserialize} and the type accepted by {@link #combine}. However, it is *not* necessarily the same type * returned by {@link #finalizeComputation}. * - * Refer to the {@link ValueType} javadocs for details on the implications of choosing a type. + * Refer to the {@link ColumnType} javadocs for details on the implications of choosing a type. */ - public abstract ValueType getType(); + public abstract ColumnType getType(); /** * Get the type for the final form of this this aggregator, i.e. the type of the value returned by * {@link #finalizeComputation}. This may be the same as or different than the types expected in {@link #deserialize} * and {@link #combine}. * - * Refer to the {@link ValueType} javadocs for details on the implications of choosing a type. + * Refer to the {@link ColumnType} javadocs for details on the implications of choosing a type. */ - public abstract ValueType getFinalizedType(); - - /** - * Get the complex type name of the intermediate type for this aggregator. - * - * This should ONLY be implemented if the type is complex (i.e. not a simple, numeric {@link ValueType}), and there - * must be a corresponding {@link org.apache.druid.segment.serde.ComplexMetricSerde} which was registered with - * {@link org.apache.druid.segment.serde.ComplexMetrics#registerSerde} using this type name. - * - * If you need a ValueType enum corresponding to this aggregator, use {@link #getType} instead. - * - * @throws IllegalStateException if getType() != ValueType.COMPLEX - */ - public String getComplexTypeName() - { - throw new ISE("Complex type name not is not available for %s of type %s", getName(), getType()); - } + public abstract ColumnType getFinalizedType(); /** * Returns the maximum size that this aggregator will require in bytes for intermediate storage of results. diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index cb91b5b3c33c..f5d08abcadea 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -35,7 +35,6 @@ import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.segment.virtual.ExpressionSelectors; @@ -338,7 +337,7 @@ public static boolean canVectorize( { if (fieldName != null) { final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); - return capabilities == null || ValueType.isNumeric(capabilities.getType()); + return capabilities == null || capabilities.isNumeric(); } if (expression != null) { return fieldExpression.get().canVectorize(columnInspector); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java index 95002d542150..4702efeaba5b 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java @@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -136,15 +136,15 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java index 273c3b765b93..686288af7e26 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; 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.InputBindings; import org.apache.druid.math.expr.Parser; import org.apache.druid.math.expr.SettableObjectBinding; @@ -44,7 +44,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionPlan; import org.apache.druid.segment.virtual.ExpressionPlanner; import org.apache.druid.segment.virtual.ExpressionSelectors; @@ -285,7 +285,7 @@ public Comparator getComparator() return (o1, o2) -> compareExpr.eval(compareBindings.get().withBinding(COMPARE_O1, o1).withBinding(COMPARE_O2, o2)).asInt(); } - switch (initialValue.get().type()) { + switch (initialValue.get().type().getType()) { case LONG: return LongSumAggregator.COMPARATOR; case DOUBLE: @@ -373,27 +373,27 @@ public List getRequiredColumns() } @Override - public ValueType getType() + public ColumnType getType() { if (fields == null) { - return ExprType.toValueType(initialCombineValue.get().type()); + return ExpressionType.toColumnType(initialCombineValue.get().type()); } - return ExprType.toValueType(initialValue.get().type()); + return ExpressionType.toColumnType(initialValue.get().type()); } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { Expr finalizeExpr = finalizeExpression.get(); ExprEval initialVal = initialCombineValue.get(); if (finalizeExpr != null) { - ExprType type = finalizeExpr.getOutputType(finalizeInspector.get()); + ExpressionType type = finalizeExpr.getOutputType(finalizeInspector.get()); if (type == null) { type = initialVal.type(); } - return ExprType.toValueType(type); + return ExpressionType.toColumnType(type); } - return ExprType.toValueType(initialVal.type()); + return ExpressionType.toColumnType(initialVal.type()); } @Override @@ -525,14 +525,14 @@ private ColumnInspector inspectorWithAccumulator(ColumnInspector inspector) public ColumnCapabilities getColumnCapabilities(String column) { if (accumulatorId.equals(column)) { - return ColumnCapabilitiesImpl.createDefault().setType(ExprType.toValueType(initialValue.get().type())); + return ColumnCapabilitiesImpl.createDefault().setType(ExpressionType.toColumnType(initialValue.get().type())); } return inspector.getColumnCapabilities(column); } @Nullable @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { if (accumulatorId.equals(name)) { return initialValue.get().type(); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java index 852af4f1a1ba..17b4841c58da 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.joda.time.Interval; @@ -187,19 +187,13 @@ public byte[] getCacheKey() } @Override - public String getComplexTypeName() - { - return delegate.getComplexTypeName(); - } - - @Override - public ValueType getType() + public ColumnType getType() { return delegate.getType(); } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return delegate.getFinalizedType(); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java index 62fbb47aa095..08a6a800a7ff 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/GroupingAggregatorFactory.java @@ -30,7 +30,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.utils.CollectionUtils; @@ -207,15 +207,15 @@ public List requiredFields() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java index 724fe87dd83c..cb919543c958 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/HistogramAggregatorFactory.java @@ -25,7 +25,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -37,6 +37,10 @@ public class HistogramAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("histogram"); + // todo: this isn't registered with serde, is it a lie? should we just report null as the type name? or register a serde? + public static final ColumnType TYPE_VISUAL = ColumnType.ofComplex("histogramVisual"); + private final String name; private final String fieldName; private final List breaksList; @@ -201,28 +205,22 @@ public byte[] getCacheKey() return buf.array(); } - @Override - public String getComplexTypeName() - { - return "histogram"; - } - /** * actual type is {@link Histogram} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } /** * actual type is {@link HistogramVisual} */ @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.COMPLEX; + return TYPE_VISUAL; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java index ef3c151aa16b..4f889eb42fc7 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/JavaScriptAggregatorFactory.java @@ -33,7 +33,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.checkerframework.checker.nullness.qual.EnsuresNonNull; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; @@ -268,15 +268,15 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java index 551717d2e257..63f9e0dce236 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java @@ -26,7 +26,7 @@ import org.apache.druid.segment.BaseNullableColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -148,7 +148,7 @@ protected VectorAggregator factorizeVector( } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return getType(); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/PostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/PostAggregator.java index 2996a18197e4..61e85e44f99f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/PostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/PostAggregator.java @@ -21,7 +21,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.Cacheable; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -45,11 +45,11 @@ public interface PostAggregator extends Cacheable String getName(); /** - * Return the output type of a row processed with this post aggregator. Refer to the {@link ValueType} javadocs + * Return the output type of a row processed with this post aggregator. Refer to the {@link ColumnType} javadocs * for details on the implications of choosing a type. */ @Nullable - ValueType getType(); + ColumnType getType(); /** * Allows returning an enriched post aggregator, built from contextual information available from the given map of diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java index bd44361e38ec..6ff7a865aba6 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java @@ -32,6 +32,8 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -132,7 +134,7 @@ private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory col { if (fieldName != null) { ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return capabilities != null && capabilities.getType() == ValueType.STRING; + return Types.is(capabilities, ValueType.STRING); } return false; } @@ -148,12 +150,12 @@ public Object deserialize(Object object) } @Override - public ValueType getType() + public ColumnType getType() { if (storeDoubleAsFloat) { - return ValueType.FLOAT; + return ColumnType.FLOAT; } - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java index 03b9f923da6c..c5e06fa0af9d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java @@ -31,6 +31,8 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -129,9 +131,9 @@ public Object deserialize(Object object) } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override @@ -232,7 +234,7 @@ private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory col { if (fieldName != null) { ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return capabilities != null && capabilities.getType() == ValueType.STRING; + return Types.is(capabilities, ValueType.STRING); } return false; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java index bf297e1d4e93..27417d772fea 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java @@ -31,6 +31,8 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -132,9 +134,9 @@ public Object deserialize(Object object) @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override @@ -235,7 +237,7 @@ private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory col { if (fieldName != null) { ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return capabilities != null && capabilities.getType() == ValueType.STRING; + return Types.is(capabilities, ValueType.STRING); } return false; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java index 3332e959def7..19650012b4bf 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java @@ -24,7 +24,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -139,19 +139,13 @@ public List requiredFields() } @Override - public String getComplexTypeName() - { - return delegate.getComplexTypeName(); - } - - @Override - public ValueType getType() + public ColumnType getType() { return delegate.getType(); } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return delegate.getFinalizedType(); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java index d18f3eecfa03..53ac5637291d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -120,7 +120,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName); - if (capabilities == null || capabilities.getType().isNumeric()) { + if (capabilities == null || capabilities.isNumeric()) { return new DoubleAnyVectorAggregator(selectorFactory.makeValueSelector(fieldName)); } else { return NumericNilVectorAggregator.doubleNilVectorAggregator(); @@ -209,13 +209,13 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return storeDoubleAsFloat ? ValueType.FLOAT : ValueType.DOUBLE; + return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return getType(); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java index b2a4f8e12056..b06ee6cc3b8f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactory.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -117,7 +117,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName); - if (capabilities == null || capabilities.getType().isNumeric()) { + if (capabilities == null || capabilities.isNumeric()) { return new FloatAnyVectorAggregator(selectorFactory.makeValueSelector(fieldName)); } else { return NumericNilVectorAggregator.floatNilVectorAggregator(); @@ -207,15 +207,15 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java index 9826570d1212..13da125ce7c5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactory.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -116,7 +116,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory) { ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName); - if (capabilities == null || capabilities.getType().isNumeric()) { + if (capabilities == null || capabilities.isNumeric()) { return new LongAnyVectorAggregator(selectorFactory.makeValueSelector(fieldName)); } else { return NumericNilVectorAggregator.longNilVectorAggregator(); @@ -205,15 +205,15 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java index 3d5213135865..dfa20831a8e3 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/any/StringAnyAggregatorFactory.java @@ -33,7 +33,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -178,15 +178,15 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java index 57c5c7e73068..d02fc7d842ef 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorFactory.java @@ -48,7 +48,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -61,6 +61,8 @@ public class CardinalityAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("hyperUnique"); + private static List makeRequiredFieldNamesFromFields(List fields) { return ImmutableList.copyOf( @@ -315,25 +317,19 @@ public byte[] getCacheKey() .build(); } - @Override - public String getComplexTypeName() - { - return "hyperUnique"; - } - /** * actual type is {@link HyperLogLogCollector} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return round ? ValueType.LONG : ValueType.DOUBLE; + return round ? ColumnType.LONG : ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java index 51d025988c37..12ed3c5f04b4 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/types/CardinalityAggregatorColumnSelectorStrategyFactory.java @@ -23,7 +23,6 @@ import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; public class CardinalityAggregatorColumnSelectorStrategyFactory implements ColumnSelectorStrategyFactory @@ -34,8 +33,7 @@ public CardinalityAggregatorColumnSelectorStrategy makeColumnSelectorStrategy( ColumnValueSelector selector ) { - ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case STRING: return new StringCardinalityAggregatorColumnSelectorStrategy(); case LONG: @@ -45,7 +43,7 @@ public CardinalityAggregatorColumnSelectorStrategy makeColumnSelectorStrategy( case DOUBLE: return new DoubleCardinalityAggregatorColumnSelectorStrategy(); default: - throw new IAE("Cannot create query type helper from invalid type [%s]", type); + throw new IAE("Cannot create query type helper from invalid type [%s]", capabilities.asTypeString()); } } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java index 15463cf1d938..102f7a864fe2 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/DoubleFirstAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -274,18 +274,18 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return storeDoubleAsFloat ? ValueType.FLOAT : ValueType.DOUBLE; + return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { // this is a copy of getComplexTypeName in the hopes that someday groupby v1 is no more and it will report it's actual // type of COMPLEX - return storeDoubleAsFloat ? ValueType.FLOAT : ValueType.DOUBLE; + return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java index ab9211f1277c..64b5c65497dc 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/FloatFirstAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -271,16 +271,16 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java index 697663c38574..90dd6f5025dd 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/LongFirstAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -269,16 +269,16 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java index 32a35433da1c..51b04c82ca7b 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstAggregatorFactory.java @@ -36,7 +36,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -50,6 +50,8 @@ @JsonTypeName("stringFirst") public class StringFirstAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("serializablePairLongString"); + private static final Aggregator NIL_AGGREGATOR = new StringFirstAggregator( NilColumnValueSelector.instance(), NilColumnValueSelector.instance(), @@ -257,25 +259,19 @@ public byte[] getCacheKey() .build(); } - @Override - public String getComplexTypeName() - { - return "serializablePairLongString"; - } - /** * actual type is {@link SerializablePairLongString} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java index 910cb9433581..323ce413e6fc 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/first/StringFirstLastUtils.java @@ -43,7 +43,7 @@ public static boolean selectorNeedsFoldCheck( @Nullable final ColumnCapabilities valueSelectorCapabilities ) { - if (valueSelectorCapabilities != null && valueSelectorCapabilities.getType() != ValueType.COMPLEX) { + if (valueSelectorCapabilities != null && !valueSelectorCapabilities.is(ValueType.COMPLEX)) { // Known, non-complex type. return false; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java index 103917678c03..301eff3b3d60 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.PostAggregatorIds; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -104,7 +104,7 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { return aggregatorFactory != null ? aggregatorFactory.getFinalizedType() diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index bdd3d59a5534..a37ec120f887 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -42,6 +42,8 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -56,6 +58,9 @@ */ public class HyperUniquesAggregatorFactory extends AggregatorFactory { + public static final ColumnType PRECOMPUTED_TYPE = ColumnType.ofComplex("preComputedHyperUnique"); + public static final ColumnType TYPE = ColumnType.ofComplex("hyperUnique"); + public static Object estimateCardinality(@Nullable Object object, boolean round) { final HyperLogLogCollector collector = (HyperLogLogCollector) object; @@ -129,7 +134,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory) { final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName); - if (capabilities == null || capabilities.getType() != ValueType.COMPLEX) { + if (!Types.is(capabilities, ValueType.COMPLEX)) { return NoopVectorAggregator.instance(); } else { return new HyperUniquesVectorAggregator(selectorFactory.makeObjectSelector(fieldName)); @@ -258,30 +263,19 @@ public byte[] getCacheKey() .appendBoolean(round) .build(); } - - @Override - public String getComplexTypeName() - { - if (isInputHyperUnique) { - return "preComputedHyperUnique"; - } else { - return "hyperUnique"; - } - } - /** * actual type is {@link HyperLogLogCollector} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return isInputHyperUnique ? PRECOMPUTED_TYPE : TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return round ? ValueType.LONG : ValueType.DOUBLE; + return round ? ColumnType.LONG : ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java index 11a7e8d4b961..1113290ce1c9 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/DoubleLastAggregatorFactory.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -272,18 +272,18 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return storeDoubleAsFloat ? ValueType.FLOAT : ValueType.DOUBLE; + return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { // this is a copy of getComplexTypeName in the hopes that someday groupby v1 is no more and it will report it's actual // type - return storeDoubleAsFloat ? ValueType.FLOAT : ValueType.DOUBLE; + return storeDoubleAsFloat ? ColumnType.FLOAT : ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java index 01d7808c830b..63e348ee2a15 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/FloatLastAggregatorFactory.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -269,16 +269,16 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.FLOAT; + return ColumnType.FLOAT; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java index 5a8964dc3da2..8809d9ab9b78 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/LongLastAggregatorFactory.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -267,16 +267,16 @@ public byte[] getCacheKey() } @Override - public ValueType getType() + public ColumnType getType() { // if we don't pretend to be a primitive, group by v1 gets sad and doesn't work because no complex type serde - return ValueType.LONG; + return ColumnType.LONG; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java index 29543518f66c..4f337682443f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/last/StringLastAggregatorFactory.java @@ -38,7 +38,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -51,6 +51,8 @@ @JsonTypeName("stringLast") public class StringLastAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("serializablePairLongString"); + private static final Aggregator NIL_AGGREGATOR = new StringLastAggregator( NilColumnValueSelector.instance(), NilColumnValueSelector.instance(), @@ -214,25 +216,19 @@ public byte[] getCacheKey() .build(); } - @Override - public String getComplexTypeName() - { - return "serializablePairLongString"; - } - /** * actual type is {@link SerializablePairLongString} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java index 51ad33aaeaa7..72df213c801d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -46,6 +46,7 @@ */ public class DoubleMeanAggregatorFactory extends AggregatorFactory { + public static final ColumnType TYPE = ColumnType.ofComplex("doubleMean"); private final String name; private final String fieldName; @@ -78,25 +79,19 @@ public List requiredFields() return Collections.singletonList(fieldName); } - @Override - public String getComplexTypeName() - { - return "doubleMean"; - } - /** * actual type is {@link DoubleMeanHolder} */ @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; + return TYPE; } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override @@ -127,7 +122,7 @@ public VectorAggregator factorizeVector(final VectorColumnSelectorFactory select public boolean canVectorize(ColumnInspector columnInspector) { final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); - return capabilities == null || capabilities.getType().isNumeric(); + return capabilities == null || capabilities.isNumeric(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregator.java index fb53f616e4e3..a8007fa1003f 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregator.java @@ -29,10 +29,9 @@ import org.apache.druid.query.aggregation.DoubleSumAggregator; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; - import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -135,9 +134,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/ConstantPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/ConstantPostAggregator.java index 8651f7f749ec..6f49727f5334 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/ConstantPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/ConstantPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.HashSet; @@ -76,9 +76,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return constantValue instanceof Long ? ValueType.LONG : ValueType.DOUBLE; + return constantValue instanceof Long ? ColumnType.LONG : ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregator.java index 78ef821d8ce2..4d28de61077c 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.HashSet; @@ -101,9 +101,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregator.java index 6321a5370c85..ae343c7d4898 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.HashSet; @@ -101,9 +101,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java index cab72bd20de8..5fecc98cc9f0 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/ExpressionPostAggregator.java @@ -36,7 +36,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; @@ -68,7 +68,7 @@ public class ExpressionPostAggregator implements PostAggregator // type is ignored from equals and friends because it is computed by decorate, and all post-aggs should be decorated // prior to usage (and is currently done so in the query constructors of all queries which can have post-aggs) @Nullable - private final ValueType outputType; + private final ColumnType outputType; private final ExprMacroTable macroTable; private final Map> finalizers; @@ -123,7 +123,7 @@ private ExpressionPostAggregator( final String name, final String expression, @Nullable final String ordering, - @Nullable final ValueType outputType, + @Nullable final ColumnType outputType, final ExprMacroTable macroTable, final Map> finalizers, final Supplier parsed, @@ -189,7 +189,7 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { // computed by decorate return outputType; @@ -245,7 +245,7 @@ public enum Ordering implements Comparator * Ensures the following order: numeric > NaN > Infinite. * * The name may be referenced via Ordering.valueOf(String) in the constructor {@link - * ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ValueType, ExprMacroTable, Map, Supplier, Supplier)}. + * ExpressionPostAggregator#ExpressionPostAggregator(String, String, String, ColumnType, ExprMacroTable, Map, Supplier, Supplier)}. */ @SuppressWarnings("unused") numericFirst { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregator.java index 23c8ee9418d9..2593567334eb 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -43,7 +43,7 @@ public class FieldAccessPostAggregator implements PostAggregator // type is ignored from equals and friends because it is computed by decorate, and all post-aggs should be decorated // prior to usage (and is currently done so in the query constructors of all queries which can have post-aggs) @Nullable - private final ValueType type; + private final ColumnType type; @JsonCreator public FieldAccessPostAggregator( @@ -54,7 +54,7 @@ public FieldAccessPostAggregator( this(name, fieldName, null); } - private FieldAccessPostAggregator(@Nullable String name, String fieldName, @Nullable ValueType type) + private FieldAccessPostAggregator(@Nullable String name, String fieldName, @Nullable ColumnType type) { Preconditions.checkNotNull(fieldName); this.name = name; @@ -89,7 +89,7 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { return type; } @@ -97,7 +97,7 @@ public ValueType getType() @Override public FieldAccessPostAggregator decorate(Map aggregators) { - final ValueType type; + final ColumnType type; if (aggregators != null && aggregators.containsKey(fieldName)) { type = aggregators.get(fieldName).getType(); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregator.java index e3c4f41572b6..70bf6cad3b54 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregator.java @@ -26,7 +26,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Comparator; @@ -42,7 +42,7 @@ public class FinalizingFieldAccessPostAggregator implements PostAggregator // type is ignored from equals and friends because it is computed by decorate, and all post-aggs should be decorated // prior to usage (and is currently done so in the query constructors of all queries which can have post-aggs) @Nullable - private final ValueType finalizedType; + private final ColumnType finalizedType; @Nullable private final Comparator comparator; @Nullable @@ -60,7 +60,7 @@ public FinalizingFieldAccessPostAggregator( private FinalizingFieldAccessPostAggregator( final String name, final String fieldName, - @Nullable final ValueType finalizedType, + @Nullable final ColumnType finalizedType, @Nullable final Comparator comparator, @Nullable final Function finalizer ) @@ -106,7 +106,7 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { return finalizedType; } @@ -116,7 +116,7 @@ public FinalizingFieldAccessPostAggregator decorate(final Map theComparator; final Function theFinalizer; - final ValueType finalizedType; + final ColumnType finalizedType; if (aggregators != null && aggregators.containsKey(fieldName)) { //noinspection unchecked diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java index b19c59f83677..dd20b7320d42 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregator.java @@ -29,7 +29,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregator; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.checkerframework.checker.nullness.qual.EnsuresNonNull; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.mozilla.javascript.Context; @@ -176,9 +176,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregator.java index 3e57c1690059..2c7a10ca2c74 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.HashSet; @@ -101,9 +101,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregator.java index 5353e195bb71..be8cc6d4bfe9 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregator.java @@ -27,7 +27,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Comparator; import java.util.HashSet; @@ -101,9 +101,9 @@ public String getName() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/dimension/BaseFilteredDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/BaseFilteredDimensionSpec.java index 5fae72f961e4..163436075803 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/BaseFilteredDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/BaseFilteredDimensionSpec.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; /** */ @@ -56,7 +56,7 @@ public String getOutputName() } @Override - public ValueType getOutputType() + public ColumnType getOutputType() { return delegate.getOutputType(); } diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java index b1c740f77c4e..0a869a92271a 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/DefaultDimensionSpec.java @@ -24,7 +24,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -44,17 +44,17 @@ public static DefaultDimensionSpec of(String dimensionName) private static final byte CACHE_TYPE_ID = 0x0; private final String dimension; private final String outputName; - private final ValueType outputType; + private final ColumnType outputType; @JsonCreator public DefaultDimensionSpec( @JsonProperty("dimension") String dimension, @JsonProperty("outputName") @Nullable String outputName, - @JsonProperty("outputType") @Nullable ValueType outputType + @JsonProperty("outputType") @Nullable ColumnType outputType ) { this.dimension = dimension; - this.outputType = outputType == null ? ValueType.STRING : outputType; + this.outputType = outputType == null ? ColumnType.STRING : outputType; // Do null check for legacy backwards compatibility, callers should be setting the value. this.outputName = outputName == null ? dimension : outputName; @@ -62,7 +62,7 @@ public DefaultDimensionSpec( public DefaultDimensionSpec(String dimension, String outputName) { - this(dimension, outputName, ValueType.STRING); + this(dimension, outputName, ColumnType.STRING); } @Override @@ -81,7 +81,7 @@ public String getOutputName() @Override @JsonProperty - public ValueType getOutputType() + public ColumnType getOutputType() { return outputType; } diff --git a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java index 5d245b2fe408..541e6d8da28c 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/DimensionSpec.java @@ -26,7 +26,7 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -51,7 +51,7 @@ public interface DimensionSpec extends Cacheable String getOutputName(); - ValueType getOutputType(); + ColumnType getOutputType(); //ExtractionFn can be implemented with decorate(..) fn @Deprecated diff --git a/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java index 5f84f8d6ee5b..e2a1956f0a0f 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/ExtractionDimensionSpec.java @@ -25,7 +25,7 @@ import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; /** */ @@ -36,13 +36,13 @@ public class ExtractionDimensionSpec implements DimensionSpec private final String dimension; private final ExtractionFn extractionFn; private final String outputName; - private final ValueType outputType; + private final ColumnType outputType; @JsonCreator public ExtractionDimensionSpec( @JsonProperty("dimension") String dimension, @JsonProperty("outputName") String outputName, - @JsonProperty("outputType") ValueType outputType, + @JsonProperty("outputType") ColumnType outputType, @JsonProperty("extractionFn") ExtractionFn extractionFn, // for backwards compatibility @Deprecated @JsonProperty("dimExtractionFn") ExtractionFn dimExtractionFn @@ -53,7 +53,7 @@ public ExtractionDimensionSpec( this.dimension = dimension; this.extractionFn = extractionFn != null ? extractionFn : dimExtractionFn; - this.outputType = outputType == null ? ValueType.STRING : outputType; + this.outputType = outputType == null ? ColumnType.STRING : outputType; // Do null check for backwards compatibility this.outputName = outputName == null ? dimension : outputName; @@ -64,7 +64,7 @@ public ExtractionDimensionSpec(String dimension, String outputName, ExtractionFn this(dimension, outputName, null, extractionFn, null); } - public ExtractionDimensionSpec(String dimension, String outputName, ValueType outputType, ExtractionFn extractionFn) + public ExtractionDimensionSpec(String dimension, String outputName, ColumnType outputType, ExtractionFn extractionFn) { this(dimension, outputName, outputType, extractionFn, null); } @@ -85,7 +85,7 @@ public String getOutputName() @Override @JsonProperty - public ValueType getOutputType() + public ColumnType getOutputType() { return outputType; } diff --git a/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java b/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java index 69c3b91621dc..35e6e6cd200f 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/LookupDimensionSpec.java @@ -33,7 +33,7 @@ import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -114,10 +114,10 @@ public String getOutputName() } @Override - public ValueType getOutputType() + public ColumnType getOutputType() { // Extraction functions always output String - return ValueType.STRING; + return ColumnType.STRING; } @JsonProperty diff --git a/processing/src/main/java/org/apache/druid/query/expression/ContainsExpr.java b/processing/src/main/java/org/apache/druid/query/expression/ContainsExpr.java index 240f3b7b6d4c..59b6d6d5ce7b 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/ContainsExpr.java +++ b/processing/src/main/java/org/apache/druid/query/expression/ContainsExpr.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -72,9 +72,9 @@ public ExprEval eval(final Expr.ObjectBinding bindings) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java index 2b948d5e6986..96c5937b6848 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -89,7 +89,7 @@ public ExprEval eval(final ObjectBinding bindings) { ExprEval eval = arg.eval(bindings); boolean match; - switch (eval.type()) { + switch (eval.type().getType()) { case STRING: match = isStringMatch(eval.asString()); break; @@ -121,9 +121,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java index 3cade2f463c8..7be011b9a4f7 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java @@ -23,7 +23,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -78,7 +78,7 @@ private IPv4AddressParseExpr(Expr arg) public ExprEval eval(final ObjectBinding bindings) { ExprEval eval = arg.eval(bindings); - switch (eval.type()) { + switch (eval.type().getType()) { case STRING: return evalAsString(eval); case LONG: @@ -97,9 +97,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java index 8d2a661ee6ad..6d7c5c581418 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java @@ -23,7 +23,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -77,7 +77,7 @@ private IPv4AddressStringifyExpr(Expr arg) public ExprEval eval(final ObjectBinding bindings) { ExprEval eval = arg.eval(bindings); - switch (eval.type()) { + switch (eval.type().getType()) { case STRING: return evalAsString(eval); case LONG: @@ -96,9 +96,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java index 6d7246229e12..a72acab8d501 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; 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.query.filter.LikeDimFilter; import javax.annotation.Nonnull; @@ -94,9 +94,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java index 9b25e5de62a7..88341f2e8529 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java @@ -26,7 +26,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; 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.Exprs; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; @@ -100,9 +100,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java index ab76612ee345..c0dcc395ed39 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -101,9 +101,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java index 7701341046f6..fbcdf113aaca 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -93,9 +93,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java index 574daba523c2..9e137f2ffe51 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java @@ -26,7 +26,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -95,9 +95,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -163,9 +163,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java index be2856e17e5d..d2c7530b9d89 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.chrono.ISOChronology; @@ -166,14 +166,14 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { switch (unit) { case CENTURY: case MILLENNIUM: - return ExprType.DOUBLE; + return ExpressionType.DOUBLE; default: - return ExprType.LONG; + return ExpressionType.LONG; } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java index 5fb85c05df3c..1b9fa585dc87 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java @@ -24,7 +24,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; 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.vector.CastToTypeVectorProcessor; import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.math.expr.vector.LongOutLongInFunctionVectorProcessor; @@ -117,9 +117,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override @@ -133,7 +133,7 @@ public ExprVectorProcessor buildVectorized(VectorInputBindingInspector in { ExprVectorProcessor processor; processor = new LongOutLongInFunctionVectorProcessor( - CastToTypeVectorProcessor.cast(args.get(0).buildVectorized(inspector), ExprType.LONG), + CastToTypeVectorProcessor.cast(args.get(0).buildVectorized(inspector), ExpressionType.LONG), inspector.getMaxVectorSize() ) { @@ -194,9 +194,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java index 58dad99f20c8..f217bfa80a75 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -101,9 +101,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java index fff0739f5118..0b4940351991 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java @@ -25,7 +25,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; @@ -104,9 +104,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java index de1293aef005..765ad8947d0a 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java @@ -24,7 +24,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.joda.time.Chronology; import org.joda.time.Period; import org.joda.time.chrono.ISOChronology; @@ -115,9 +115,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } @@ -151,9 +151,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.LONG; + return ExpressionType.LONG; } } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java index a6e5e779fa42..f07807e32b68 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java @@ -26,7 +26,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -172,9 +172,9 @@ public Expr visit(Shuttle shuttle) @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override @@ -300,9 +300,9 @@ public BindingAnalysis analyzeInputs() @Nullable @Override - public ExprType getOutputType(InputBindingInspector inspector) + public ExpressionType getOutputType(InputBindingInspector inspector) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java index b95ebc0c5ff0..93826c729174 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java @@ -92,7 +92,7 @@ public VectorValueMatcherFactory makeObjectProcessor( final VectorObjectSelector selector ) { - if (ValueType.STRING.equals(capabilities.getType())) { + if (capabilities.is(ValueType.STRING)) { return new StringObjectVectorValueMatcher(selector); } return new ObjectVectorValueMatcher(selector); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 4df7d49e8c20..43328c1a0db8 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -65,8 +65,8 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -549,7 +549,7 @@ private Ordering getRowOrderingForPushDown( final IntList orderedFieldNumbers = new IntArrayList(); final Set dimsInOrderBy = new HashSet<>(); final List needsReverseList = new ArrayList<>(); - final List dimensionTypes = new ArrayList<>(); + final List dimensionTypes = new ArrayList<>(); final List comparators = new ArrayList<>(); for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { @@ -560,7 +560,7 @@ private Ordering getRowOrderingForPushDown( orderedFieldNumbers.add(resultRowSignature.indexOf(dim.getOutputName())); dimsInOrderBy.add(dimIndex); needsReverseList.add(needsReverse); - final ValueType type = dimensions.get(dimIndex).getOutputType(); + final ColumnType type = dimensions.get(dimIndex).getOutputType(); dimensionTypes.add(type); comparators.add(orderSpec.getDimensionComparator()); } @@ -570,7 +570,7 @@ private Ordering getRowOrderingForPushDown( if (!dimsInOrderBy.contains(i)) { orderedFieldNumbers.add(resultRowSignature.indexOf(dimensions.get(i).getOutputName())); needsReverseList.add(false); - final ValueType type = dimensions.get(i).getOutputType(); + final ColumnType type = dimensions.get(i).getOutputType(); dimensionTypes.add(type); comparators.add(StringComparators.LEXICOGRAPHIC); } @@ -736,7 +736,7 @@ private DateTime computeUniversalTimestamp() private static int compareDimsForLimitPushDown( final IntList fields, final List needsReverseList, - final List dimensionTypes, + final List dimensionTypes, final List comparators, final ResultRow lhs, final ResultRow rhs @@ -745,13 +745,13 @@ private static int compareDimsForLimitPushDown( for (int i = 0; i < fields.size(); i++) { final int fieldNumber = fields.getInt(i); final StringComparator comparator = comparators.get(i); - final ValueType dimensionType = dimensionTypes.get(i); + final ColumnType dimensionType = dimensionTypes.get(i); final int dimCompare; final Object lhsObj = lhs.get(fieldNumber); final Object rhsObj = rhs.get(fieldNumber); - if (ValueType.isNumeric(dimensionType)) { + if (dimensionType.isNumeric()) { if (comparator.equals(StringComparators.NUMERIC)) { dimCompare = DimensionHandlerUtils.compareObjectsAsType(lhsObj, rhsObj, dimensionType); } else { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java index 3a34b76c874a..81ea974d11c1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryEngine.java @@ -325,7 +325,7 @@ public RowIterator(GroupByQuery query, final Cursor cursor, ByteBuffer metricsBu dimNames = Lists.newArrayListWithExpectedSize(dimensionSpecs.size()); for (final DimensionSpec dimSpec : dimensionSpecs) { - if (dimSpec.getOutputType() != ValueType.STRING) { + if (!dimSpec.getOutputType().is(ValueType.STRING)) { throw new UnsupportedOperationException( "GroupBy v1 only supports dimensions with an outputType of STRING." ); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index 6f1cee955d1f..f0d834948199 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -63,6 +63,8 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; @@ -298,7 +300,7 @@ public static int getCardinalityForArrayAggregation( } // Choose array-based aggregation if the grouping key is a single string dimension of a known cardinality - if (columnCapabilities != null && columnCapabilities.getType().equals(ValueType.STRING) && cardinality > 0) { + if (Types.is(columnCapabilities, ValueType.STRING) && cardinality > 0) { final AggregatorFactory[] aggregatorFactories = query.getAggregatorSpecs().toArray(new AggregatorFactory[0]); final long requiredBufferCapacity = BufferArrayGrouper.requiredBufferCapacity( cardinality, @@ -351,7 +353,7 @@ public static void convertRowTypesToOutputTypes( for (int i = 0; i < dimensionSpecs.size(); i++) { DimensionSpec dimSpec = dimensionSpecs.get(i); final int resultRowIndex = resultRowDimensionStart + i; - final ValueType outputType = dimSpec.getOutputType(); + final ColumnType outputType = dimSpec.getOutputType(); resultRow.set( resultRowIndex, @@ -368,7 +370,7 @@ public static boolean canPushDownLimit(ColumnSelectorFactory columnSelectorFacto ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName); if (capabilities != null) { // strings can be pushed down if dictionaries are sorted and unique per id - if (capabilities.getType() == ValueType.STRING) { + if (capabilities.is(ValueType.STRING)) { return capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue(); } // party on @@ -387,8 +389,7 @@ public GroupByColumnSelectorStrategy makeColumnSelectorStrategy( ColumnValueSelector selector ) { - ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case STRING: DimensionSelector dimSelector = (DimensionSelector) selector; if (dimSelector.getValueCardinality() >= 0) { @@ -403,7 +404,7 @@ public GroupByColumnSelectorStrategy makeColumnSelectorStrategy( case DOUBLE: return makeNullableNumericStrategy(new DoubleGroupByColumnSelectorStrategy()); default: - throw new IAE("Cannot create query type helper from invalid type [%s]", type); + throw new IAE("Cannot create query type helper from invalid type [%s]", capabilities.asTypeString()); } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GrouperBufferComparatorUtils.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GrouperBufferComparatorUtils.java index 3f0f70446c5d..41543cbab066 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GrouperBufferComparatorUtils.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GrouperBufferComparatorUtils.java @@ -28,7 +28,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -148,7 +148,7 @@ public static Grouper.BufferComparator bufferComparatorWithAggregators( int aggIndex = OrderByColumnSpec.getAggIndexForOrderBy(orderSpec, Arrays.asList(aggregatorFactories)); if (aggIndex >= 0) { final StringComparator stringComparator = orderSpec.getDimensionComparator(); - final ValueType valueType = aggregatorFactories[aggIndex].getType(); + final ColumnType valueType = aggregatorFactories[aggIndex].getType(); // Aggregators start after dimensions final int aggOffset = keySize + aggregatorOffsets[aggIndex]; @@ -324,13 +324,13 @@ private static int compareDimsInBuffersForNullFudgeTimestamp( } private static Grouper.BufferComparator makeNumericBufferComparator( - ValueType valueType, + ColumnType valueType, int keyBufferPosition, boolean pushLimitDown, @Nullable StringComparator stringComparator ) { - switch (valueType) { + switch (valueType.getType()) { case LONG: return makeBufferComparatorForLong(keyBufferPosition, pushLimitDown, stringComparator); case FLOAT: diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 8a3e2a46b6c8..dddc1ebb628d 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -67,6 +67,7 @@ import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.BooleanValueMatcher; @@ -193,7 +194,7 @@ public static Pair, Accumulator // See method-level javadoc; we go into combining mode if there is no subquery. final boolean combining = subquery == null; - final List valueTypes = DimensionHandlerUtils.getValueTypesFromDimensionSpecs(query.getDimensions()); + final List valueTypes = DimensionHandlerUtils.getValueTypesFromDimensionSpecs(query.getDimensions()); final GroupByQueryConfig querySpecificConfig = config.withOverrides(query); final boolean includeTimestamp = query.getResultRowHasTimestamp(); @@ -466,7 +467,7 @@ private static ValueExtractFunction makeValueExtractFunction( final boolean combining, final boolean includeTimestamp, final ColumnSelectorFactory columnSelectorFactory, - final List valueTypes + final List valueTypes ) { final TimestampExtractFunction timestampExtractFn = includeTimestamp ? @@ -705,8 +706,7 @@ public InputRawSupplierColumnSelectorStrategy makeColumnSelectorStrategy( ColumnValueSelector selector ) { - ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case STRING: return new StringInputRawSupplierColumnSelectorStrategy(); case LONG: @@ -719,7 +719,7 @@ public InputRawSupplierColumnSelectorStrategy makeColumnSelectorStrategy( return (InputRawSupplierColumnSelectorStrategy) columnSelector -> () -> columnSelector.isNull() ? null : columnSelector.getDouble(); default: - throw new IAE("Cannot create query type helper from invalid type [%s]", type); + throw new IAE("Cannot create query type helper from invalid type [%s]", capabilities.asTypeString()); } } } @@ -748,14 +748,14 @@ private static Supplier[] getValueSuppliersForDimensions( @SuppressWarnings("unchecked") private static Function[] makeValueConvertFunctions( - final List valueTypes + final List valueTypes ) { final Function[] functions = new Function[valueTypes.size()]; for (int i = 0; i < functions.length; i++) { // Subquery post-aggs aren't added to the rowSignature (see rowSignatureFor() in GroupByQueryHelper) because // their types aren't known, so default to String handling. - final ValueType type = valueTypes.get(i) == null ? ValueType.STRING : valueTypes.get(i); + final ColumnType type = valueTypes.get(i) == null ? ColumnType.STRING : valueTypes.get(i); functions[i] = input -> DimensionHandlerUtils.convertObjectToType(input, type); } return functions; @@ -769,14 +769,14 @@ private static class RowBasedKeySerdeFactory implements Grouper.KeySerdeFactory< private final DefaultLimitSpec limitSpec; private final List dimensions; final AggregatorFactory[] aggregatorFactories; - private final List valueTypes; + private final List valueTypes; RowBasedKeySerdeFactory( boolean includeTimestamp, boolean sortByDimsFirst, List dimensions, long maxDictionarySize, - List valueTypes, + List valueTypes, final AggregatorFactory[] aggregatorFactories, DefaultLimitSpec limitSpec ) @@ -868,7 +868,7 @@ private Comparator> objectComparatorWithAggs() final List aggFlags = new ArrayList<>(); final List comparators = new ArrayList<>(); final List fieldIndices = new ArrayList<>(); - final List fieldValueTypes = new ArrayList<>(); + final List fieldValueTypes = new ArrayList<>(); final Set orderByIndices = new HashSet<>(); for (OrderByColumnSpec orderSpec : limitSpec.getColumns()) { @@ -879,7 +879,7 @@ private Comparator> objectComparatorWithAggs() orderByIndices.add(dimIndex); needsReverses.add(needsReverse); aggFlags.add(false); - final ValueType type = dimensions.get(dimIndex).getOutputType(); + final ColumnType type = dimensions.get(dimIndex).getOutputType(); fieldValueTypes.add(type); comparators.add(orderSpec.getDimensionComparator()); } else { @@ -899,7 +899,7 @@ private Comparator> objectComparatorWithAggs() fieldIndices.add(i); aggFlags.add(false); needsReverses.add(false); - ValueType type = dimensions.get(i).getOutputType(); + ColumnType type = dimensions.get(i).getOutputType(); fieldValueTypes.add(type); if (type.isNumeric()) { comparators.add(StringComparators.NUMERIC); @@ -965,7 +965,7 @@ private Comparator> objectComparatorWithAggs() } } - private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, final List fieldTypes, int dimStart) + private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, final List fieldTypes, int dimStart) { for (int i = dimStart; i < key1.getKey().length; i++) { final int cmp; @@ -973,7 +973,7 @@ private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, final L // to double // timestamp is not present in fieldTypes since it only includes the dimensions. sort of hacky, but if timestamp // is included, dimstart will be 1, so subtract from 'i' to get correct index - if (ValueType.DOUBLE == fieldTypes.get(i - dimStart)) { + if (fieldTypes.get(i - dimStart).is(ValueType.DOUBLE)) { Object lhs = key1.getKey()[i]; Object rhs = key2.getKey()[i]; cmp = Comparators.naturalNullsFirst().compare( @@ -1001,7 +1001,7 @@ private static int compareDimsInRowsWithAggs( final List needsReverses, final List aggFlags, final List fieldIndices, - final List fieldTypes, + final List fieldTypes, final List comparators ) { @@ -1032,10 +1032,10 @@ private static int compareDimsInRowsWithAggs( final StringComparator comparator = comparators.get(i); - final ValueType fieldType = fieldTypes.get(i); + final ColumnType fieldType = fieldTypes.get(i); if (fieldType.isNumeric() && comparator.equals(StringComparators.NUMERIC)) { // use natural comparison - if (ValueType.DOUBLE == fieldType) { + if (fieldType.is(ValueType.DOUBLE)) { // sometimes doubles can become floats making the round trip from serde, make sure to coerce them both // to double cmp = Comparators.naturalNullsFirst().compare( @@ -1080,7 +1080,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde valueTypes; + private final List valueTypes; private final boolean enableRuntimeDictionaryGeneration; @@ -1104,7 +1104,7 @@ private static class RowBasedKeySerde implements Grouper.KeySerde dimensions, final long maxDictionarySize, final DefaultLimitSpec limitSpec, - final List valueTypes, + final List valueTypes, @Nullable final List dictionary ) { @@ -1312,14 +1312,14 @@ private RowBasedKeySerdeHelper[] makeSerdeHelpers( } private RowBasedKeySerdeHelper makeSerdeHelper( - ValueType valueType, + ColumnType valueType, int keyBufferPosition, boolean pushLimitDown, @Nullable StringComparator stringComparator, boolean enableRuntimeDictionaryGeneration ) { - switch (valueType) { + switch (valueType.getType()) { case STRING: if (enableRuntimeDictionaryGeneration) { return new DynamicDictionaryStringRowBasedKeySerdeHelper( @@ -1337,7 +1337,7 @@ private RowBasedKeySerdeHelper makeSerdeHelper( case LONG: case FLOAT: case DOUBLE: - return makeNullHandlingNumericserdeHelper(valueType, keyBufferPosition, pushLimitDown, stringComparator); + return makeNullHandlingNumericserdeHelper(valueType.getType(), keyBufferPosition, pushLimitDown, stringComparator); default: throw new IAE("invalid type: %s", valueType); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java index a141113bad5b..3af16ea3d0dc 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java @@ -49,7 +49,7 @@ public GroupByVectorColumnSelector makeSingleValueDimensionProcessor( ) { Preconditions.checkArgument( - ValueType.STRING == capabilities.getType(), + capabilities.is(ValueType.STRING), "groupBy dimension processors must be STRING typed" ); return new SingleValueStringGroupByVectorColumnSelector(selector); @@ -62,7 +62,7 @@ public GroupByVectorColumnSelector makeMultiValueDimensionProcessor( ) { Preconditions.checkArgument( - ValueType.STRING == capabilities.getType(), + capabilities.is(ValueType.STRING), "groupBy dimension processors must be STRING typed" ); throw new UnsupportedOperationException( @@ -112,7 +112,7 @@ public GroupByVectorColumnSelector makeObjectProcessor( final VectorObjectSelector selector ) { - if (ValueType.STRING.equals(capabilities.getType())) { + if (capabilities.is(ValueType.STRING)) { return new DictionaryBuildingSingleValueStringGroupByVectorColumnSelector(selector); } return NilGroupByVectorColumnSelector.INSTANCE; @@ -135,7 +135,7 @@ public GroupByVectorColumnSelector makeObjectProcessor( public boolean useDictionaryEncodedSelector(ColumnCapabilities capabilities) { Preconditions.checkArgument(capabilities != null, "Capabilities must not be null"); - Preconditions.checkArgument(capabilities.getType() == ValueType.STRING, "Must only be called on a STRING column"); + Preconditions.checkArgument(capabilities.is(ValueType.STRING), "Must only be called on a STRING column"); return capabilities.isDictionaryEncoded().isTrue(); } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java index 3046b98a6125..2ca0b19b399b 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpec.java @@ -44,6 +44,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; @@ -195,11 +196,11 @@ public Function, Sequence> build(final GroupByQue break; } - final ValueType columnType = getOrderByType(columnSpec, dimensions); + final ColumnType columnType = getOrderByType(columnSpec, dimensions); final StringComparator naturalComparator; - if (columnType == ValueType.STRING) { + if (columnType.is(ValueType.STRING)) { naturalComparator = StringComparators.LEXICOGRAPHIC; - } else if (ValueType.isNumeric(columnType)) { + } else if (columnType.isNumeric()) { naturalComparator = StringComparators.NUMERIC; } else { sortingNeeded = true; @@ -272,7 +273,7 @@ public LimitSpec merge(LimitSpec other) return this; } - private ValueType getOrderByType(final OrderByColumnSpec columnSpec, final List dimensions) + private ColumnType getOrderByType(final OrderByColumnSpec columnSpec, final List dimensions) { for (DimensionSpec dimSpec : dimensions) { if (columnSpec.getDimension().equals(dimSpec.getOutputName())) { diff --git a/processing/src/main/java/org/apache/druid/query/lookup/LookupSegment.java b/processing/src/main/java/org/apache/druid/query/lookup/LookupSegment.java index 6211f82b7a64..132715998c5f 100644 --- a/processing/src/main/java/org/apache/druid/query/lookup/LookupSegment.java +++ b/processing/src/main/java/org/apache/druid/query/lookup/LookupSegment.java @@ -22,8 +22,8 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowBasedSegment; +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.join.lookup.LookupColumnSelectorFactory; import org.apache.druid.timeline.SegmentId; @@ -39,8 +39,8 @@ public class LookupSegment extends RowBasedSegment> { private static final RowSignature ROW_SIGNATURE = RowSignature.builder() - .add(LookupColumnSelectorFactory.KEY_COLUMN, ValueType.STRING) - .add(LookupColumnSelectorFactory.VALUE_COLUMN, ValueType.STRING) + .add(LookupColumnSelectorFactory.KEY_COLUMN, ColumnType.STRING) + .add(LookupColumnSelectorFactory.VALUE_COLUMN, ColumnType.STRING) .build(); public LookupSegment(final String lookupName, final LookupExtractorFactory lookupExtractorFactory) diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java index 206ad2ce1bae..6e96cd971c33 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentAnalyzer.java @@ -42,9 +42,9 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ComplexColumn; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -119,8 +119,7 @@ public Map analyze(Segment segment) } final ColumnAnalysis analysis; - final ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case LONG: analysis = analyzeNumericColumn(capabilities, length, Long.BYTES); break; @@ -141,8 +140,8 @@ public Map analyze(Segment segment) analysis = analyzeComplexColumn(capabilities, columnHolder, storageAdapter.getColumnTypeName(columnName)); break; default: - log.warn("Unknown column type[%s].", type); - analysis = ColumnAnalysis.error(StringUtils.format("unknown_type_%s", type)); + log.warn("Unknown column type[%s].", capabilities.asTypeString()); + analysis = ColumnAnalysis.error(StringUtils.format("unknown_type_%s", capabilities.asTypeString())); } columns.put(columnName, analysis); @@ -151,7 +150,7 @@ public Map analyze(Segment segment) // Add time column too ColumnCapabilities timeCapabilities = adapterCapabilitesFn.apply(ColumnHolder.TIME_COLUMN_NAME); if (timeCapabilities == null) { - timeCapabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); + timeCapabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG); } columns.put( ColumnHolder.TIME_COLUMN_NAME, @@ -193,7 +192,7 @@ private ColumnAnalysis analyzeNumericColumn( } return new ColumnAnalysis( - capabilities.getType().name(), + capabilities.asTypeString(), capabilities.hasMultipleValues().isTrue(), capabilities.hasNulls().isMaybeTrue(), // if we don't know for sure, then we should plan to check for nulls size, @@ -249,7 +248,7 @@ private ColumnAnalysis analyzeStringColumn( } return new ColumnAnalysis( - capabilities.getType().name(), + capabilities.asTypeString(), capabilities.hasMultipleValues().isTrue(), capabilities.hasNulls().isMaybeTrue(), // if we don't know for sure, then we should plan to check for nulls size, @@ -327,7 +326,7 @@ public Long accumulate(Long accumulated, Cursor cursor) } return new ColumnAnalysis( - capabilities.getType().name(), + capabilities.asTypeString(), capabilities.hasMultipleValues().isTrue(), capabilities.hasNulls().isMaybeTrue(), // if we don't know for sure, then we should plan to check for nulls size, @@ -344,6 +343,8 @@ private ColumnAnalysis analyzeComplexColumn( final String typeName ) { + // serialize using asTypeString (which is also used for JSON so can easily round-trip complex type info back into ColumnType) + final String serdeTypeName = ColumnType.ofComplex(typeName).asTypeString(); try (final ComplexColumn complexColumn = columnHolder != null ? (ComplexColumn) columnHolder.getColumn() : null) { final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues().isTrue(); final boolean hasNulls = capabilities != null && capabilities.hasNulls().isMaybeTrue(); @@ -357,7 +358,16 @@ private ColumnAnalysis analyzeComplexColumn( final Function inputSizeFn = serde.inputSizeFn(); if (inputSizeFn == null) { - return new ColumnAnalysis(typeName, hasMultipleValues, hasNulls, 0, null, null, null, null); + return new ColumnAnalysis( + serdeTypeName, + hasMultipleValues, + hasNulls, + 0, + null, + null, + null, + null + ); } final int length = complexColumn.getLength(); @@ -367,7 +377,7 @@ private ColumnAnalysis analyzeComplexColumn( } return new ColumnAnalysis( - typeName, + serdeTypeName, hasMultipleValues, hasNulls, size, diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 98e47f7252b2..78a369a2b893 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -37,8 +37,8 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import java.util.List; import java.util.Map; @@ -168,11 +168,11 @@ public RowSignature resultArraySignature(final ScanQuery query) for (String columnName : query.getColumns()) { // With the Scan query we only know the columnType for virtual columns. Let's report those, at least. - final ValueType columnType; + final ColumnType columnType; final VirtualColumn virtualColumn = query.getVirtualColumns().getVirtualColumn(columnName); if (virtualColumn != null) { - columnType = virtualColumn.capabilities(columnName).getType(); + columnType = virtualColumn.capabilities(columnName).toColumnType(); } else { // Unknown type. In the future, it would be nice to have a way to fill these in. columnType = null; diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java index 62fb8512f4e4..7ba1e1ab549b 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryRunner.java @@ -44,7 +44,6 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import java.util.List; @@ -74,8 +73,7 @@ public SearchColumnSelectorStrategy makeColumnSelectorStrategy( ColumnValueSelector selector ) { - ValueType type = capabilities.getType(); - switch (type) { + switch (capabilities.getType()) { case STRING: return new StringSearchColumnSelectorStrategy(); case LONG: @@ -85,7 +83,7 @@ public SearchColumnSelectorStrategy makeColumnSelectorStrategy( case DOUBLE: return new DoubleSearchColumnSelectorStrategy(); default: - throw new IAE("Cannot create query type helper from invalid type [%s]", type); + throw new IAE("Cannot create query type helper from invalid type [%s]", capabilities.asTypeString()); } } } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index d811914b89d1..34254b67151c 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -52,8 +52,8 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.joda.time.DateTime; import java.util.Collections; @@ -415,7 +415,7 @@ public RowSignature resultArraySignature(TimeseriesQuery query) RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); rowSignatureBuilder.addTimeColumn(); if (StringUtils.isNotEmpty(query.getTimestampResultField())) { - rowSignatureBuilder.add(query.getTimestampResultField(), ValueType.LONG); + rowSignatureBuilder.add(query.getTimestampResultField(), ColumnType.LONG); } rowSignatureBuilder.addAggregators(query.getAggregatorSpecs()); rowSignatureBuilder.addPostAggregators(query.getPostAggregatorSpecs()); diff --git a/processing/src/main/java/org/apache/druid/query/topn/DimValHolder.java b/processing/src/main/java/org/apache/druid/query/topn/DimValHolder.java index b66cd0e28cf1..0a82cc3e72fc 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/DimValHolder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/DimValHolder.java @@ -20,7 +20,7 @@ package org.apache.druid.query.topn; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Map; @@ -102,7 +102,7 @@ public Builder withTopNMetricVal(Object topNMetricVal) * {@link org.apache.druid.query.dimension.DimensionSpec} associated with dimValue from the * calling TopNResultBuilder */ - public Builder withDimValue(Comparable dimValue, ValueType type) + public Builder withDimValue(Comparable dimValue, ColumnType type) { this.dimValue = DimensionHandlerUtils.convertObjectToType(dimValue, type); return this; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java index b036ba64c95c..44dad8a12f4f 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TimeExtractionTopNAlgorithm.java @@ -25,7 +25,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.HashMap; import java.util.Map; @@ -46,7 +46,7 @@ public TimeExtractionTopNAlgorithm(StorageAdapter storageAdapter, TopNQuery quer // This strategy is used for ExtractionFns on the __time column. They always return STRING, so we need to convert // from STRING to the desired output type. this.dimensionValueConverter = DimensionHandlerUtils.converterFromTypeToType( - ValueType.STRING, + ColumnType.STRING, query.getDimensionSpec().getOutputType() ); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNLexicographicResultBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNLexicographicResultBuilder.java index a789dce3d0c5..7bd816a203ca 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNLexicographicResultBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNLexicographicResultBuilder.java @@ -24,7 +24,7 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.joda.time.DateTime; import java.util.Arrays; @@ -118,7 +118,7 @@ public TopNResultBuilder addEntry( metricValues.put(aggFactoryNames[i + 7], metricVals[i + 7]); // lgtm [java/index-out-of-bounds] } - pQueue.add(new DimValHolder.Builder().withDimValue(dimValue, ValueType.STRING).withMetricValues(metricValues).build()); + pQueue.add(new DimValHolder.Builder().withDimValue(dimValue, ColumnType.STRING).withMetricValues(metricValues).build()); if (pQueue.size() > threshold) { pQueue.poll(); } @@ -135,7 +135,7 @@ public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndM if (shouldAdd(dimensionValue)) { pQueue.add( - new DimValHolder.Builder().withDimValue(dimensionValue, ValueType.STRING) + new DimValHolder.Builder().withDimValue(dimensionValue, ColumnType.STRING) .withMetricValues(dimensionAndMetricValueExtractor.getBaseObject()) .build() ); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index ad03f6ae3a8f..657bb8931fb9 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.Filters; import org.joda.time.Interval; @@ -188,11 +189,11 @@ private static boolean canUsePooledAlgorithm( return false; } - if (query.getDimensionSpec().getOutputType() != ValueType.STRING) { + if (!query.getDimensionSpec().getOutputType().is(ValueType.STRING)) { // non-string output cannot use the pooled algorith, even if the underlying selector supports it return false; } - if (capabilities != null && capabilities.getType() == ValueType.STRING) { + if (Types.is(capabilities, ValueType.STRING)) { // string columns must use the on heap algorithm unless they have the following capabilites return capabilities.isDictionaryEncoded().isTrue() && capabilities.areDictionaryValuesUnique().isTrue(); } else { diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java index eacbde0863f0..36f380043e2c 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java @@ -30,7 +30,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import java.util.HashMap; @@ -43,10 +43,10 @@ public class StringTopNColumnAggregatesProcessor implements TopNColumnAggregates private final Function> dimensionValueConverter; private HashMap, Aggregator[]> aggregatesStore; - public StringTopNColumnAggregatesProcessor(final ColumnCapabilities capabilities, final ValueType dimensionType) + public StringTopNColumnAggregatesProcessor(final ColumnCapabilities capabilities, final ColumnType dimensionType) { this.capabilities = capabilities; - this.dimensionValueConverter = DimensionHandlerUtils.converterFromTypeToType(ValueType.STRING, dimensionType); + this.dimensionValueConverter = DimensionHandlerUtils.converterFromTypeToType(ColumnType.STRING, dimensionType); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessorFactory.java index be6eb08f8913..659a976a5660 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/TopNColumnAggregatesProcessorFactory.java @@ -25,6 +25,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import java.util.function.Function; @@ -32,9 +33,9 @@ public class TopNColumnAggregatesProcessorFactory implements ColumnSelectorStrategyFactory> { - private final ValueType dimensionType; + private final ColumnType dimensionType; - public TopNColumnAggregatesProcessorFactory(final ValueType dimensionType) + public TopNColumnAggregatesProcessorFactory(final ColumnType dimensionType) { this.dimensionType = Preconditions.checkNotNull(dimensionType, "dimensionType"); } @@ -45,17 +46,15 @@ public TopNColumnAggregatesProcessor makeColumnSelectorStrategy( ColumnValueSelector selector ) { - final ValueType selectorType = capabilities.getType(); - - if (selectorType.equals(ValueType.STRING)) { + if (capabilities.is(ValueType.STRING)) { return new StringTopNColumnAggregatesProcessor(capabilities, dimensionType); - } else if (selectorType.isNumeric()) { + } else if (capabilities.isNumeric()) { final Function> converter; - final ValueType strategyType; + final ColumnType strategyType; // When the selector is numeric, we want to use NumericTopNColumnSelectorStrategy. It aggregates using // a numeric type and then converts to the desired output type after aggregating. We must be careful not to // convert to an output type that cannot represent all possible values of the input type. - if (ValueType.isNumeric(dimensionType)) { + if (dimensionType.isNumeric()) { // Return strategy that aggregates using the _output_ type, because this allows us to collapse values // properly (numeric types cannot always represent all values of other numeric types). converter = DimensionHandlerUtils.converterFromTypeToType(dimensionType, dimensionType); @@ -64,10 +63,10 @@ public TopNColumnAggregatesProcessor makeColumnSelectorStrategy( // Return strategy that aggregates using the _input_ type. Here we are assuming that the output type can // represent all possible values of the input type. This will be true for STRING, which is the only // non-numeric type currently supported. - converter = DimensionHandlerUtils.converterFromTypeToType(selectorType, dimensionType); - strategyType = selectorType; + converter = DimensionHandlerUtils.converterFromTypeToType(capabilities, dimensionType); + strategyType = capabilities.toColumnType(); } - switch (strategyType) { + switch (strategyType.getType()) { case LONG: return new LongTopNColumnAggregatesProcessor(converter); case FLOAT: @@ -77,6 +76,6 @@ public TopNColumnAggregatesProcessor makeColumnSelectorStrategy( } } - throw new IAE("Cannot create query type helper from invalid type [%s]", selectorType); + throw new IAE("Cannot create query type helper from invalid type [%s]", capabilities.asTypeString()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java b/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java index bf74455a2b62..d8b4aaf083fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java @@ -20,7 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nullable; @@ -39,11 +39,11 @@ public interface ColumnInspector extends Expr.InputBindingInspector @Nullable @Override - default ExprType getType(String name) + default ExpressionType getType(String name) { ColumnCapabilities capabilities = getColumnCapabilities(name); if (capabilities != null) { - return ExprType.fromValueType(capabilities.getType()); + return ExpressionType.fromColumnType(capabilities); } return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java index 4c1743787be0..1c4ca2f6f723 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java @@ -20,7 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; /** * Class that encapsulates knowledge about how to create "column processors", which are... objects that process columns @@ -45,7 +45,7 @@ public interface ColumnProcessorFactory * This allows a column processor factory to specify what type it prefers to deal with (the most 'natural' type for * whatever it is doing) when all else is equal. */ - ValueType defaultType(); + ColumnType defaultType(); /** * Create a processor for a string column. diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java index ffb06ed4e3a5..3f8edc7dad4f 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java @@ -28,6 +28,8 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.NilVectorSelector; @@ -51,7 +53,7 @@ public class ColumnProcessors * Capabilites that are used when we return a nil selector for a nonexistent column. */ public static final ColumnCapabilities NIL_COLUMN_CAPABILITIES = - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setDictionaryValuesSorted(true) @@ -120,7 +122,7 @@ public static T makeProcessor( */ public static T makeProcessor( final Expr expr, - final ValueType exprTypeHint, + final ColumnType exprTypeHint, final ColumnProcessorFactory processorFactory, final ColumnSelectorFactory selectorFactory ) @@ -215,7 +217,7 @@ private static ColumnCapabilities computeDimensionSpecCapabilities( { if (dimensionSpec.mustDecorate()) { // Decorating DimensionSpecs could do anything. We can't pass along any useful info other than the type. - return new ColumnCapabilitiesImpl().setType(ValueType.STRING); + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING); } else if (dimensionSpec.getExtractionFn() != null) { // DimensionSpec is applying an extractionFn but *not* decorating. We have some insight into how the // extractionFn will behave, so let's use it. @@ -233,7 +235,7 @@ private static ColumnCapabilities computeDimensionSpecCapabilities( } return new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(dictionaryEncoded) .setDictionaryValuesSorted(sorted && dimensionSpec.getExtractionFn().preservesOrdering()) .setDictionaryValuesUnique( @@ -270,9 +272,9 @@ private static T makeProcessorInternal( ) { final ColumnCapabilities capabilities = inputCapabilitiesFn.apply(selectorFactory); - final ValueType effectiveType = capabilities != null ? capabilities.getType() : processorFactory.defaultType(); + final TypeSignature effectiveType = capabilities != null ? capabilities : processorFactory.defaultType(); - switch (effectiveType) { + switch (effectiveType.getType()) { case STRING: return processorFactory.makeDimensionProcessor( dimensionSelectorFn.apply(selectorFactory), @@ -287,7 +289,7 @@ private static T makeProcessorInternal( case COMPLEX: return processorFactory.makeComplexProcessor(valueSelectorFunction.apply(selectorFactory)); default: - throw new ISE("Unsupported type[%s]", effectiveType); + throw new ISE("Unsupported type[%s]", effectiveType.asTypeString()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java index c2d748e5c980..8ddaabad20f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java @@ -36,12 +36,15 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import javax.annotation.Nullable; import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; @@ -55,7 +58,7 @@ public final class DimensionHandlerUtils public static final Long ZERO_LONG = 0L; public static final ColumnCapabilities DEFAULT_STRING_CAPABILITIES = - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(false) .setDictionaryValuesUnique(false) .setDictionaryValuesSorted(false) @@ -99,7 +102,7 @@ private DimensionHandlerUtils() multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling; - if (capabilities.getType() == ValueType.STRING) { + if (capabilities.is(ValueType.STRING)) { if (!capabilities.isDictionaryEncoded().isTrue()) { throw new IAE("String column must have dictionary encoding."); } @@ -111,19 +114,19 @@ private DimensionHandlerUtils() ); } - if (capabilities.getType() == ValueType.LONG) { + if (capabilities.is(ValueType.LONG)) { return new LongDimensionHandler(dimensionName); } - if (capabilities.getType() == ValueType.FLOAT) { + if (capabilities.is(ValueType.FLOAT)) { return new FloatDimensionHandler(dimensionName); } - if (capabilities.getType() == ValueType.DOUBLE) { + if (capabilities.is(ValueType.DOUBLE)) { return new DoubleDimensionHandler(dimensionName); } - if (capabilities.getType() == ValueType.COMPLEX && capabilities.getComplexTypeName() != null) { + if (capabilities.is(ValueType.COMPLEX) && capabilities.getComplexTypeName() != null) { DimensionHandlerProvider provider = DIMENSION_HANDLER_PROVIDERS.get(capabilities.getComplexTypeName()); if (provider == null) { throw new ISE("Can't find DimensionHandlerProvider for typeName [%s]", capabilities.getComplexTypeName()); @@ -135,9 +138,9 @@ private DimensionHandlerUtils() return new StringDimensionHandler(dimensionName, multiValueHandling, true, false); } - public static List getValueTypesFromDimensionSpecs(List dimSpecs) + public static List getValueTypesFromDimensionSpecs(List dimSpecs) { - List types = new ArrayList<>(dimSpecs.size()); + List types = new ArrayList<>(dimSpecs.size()); for (DimensionSpec dimSpec : dimSpecs) { types.add(dimSpec.getOutputType()); } @@ -228,7 +231,7 @@ private static ColumnValueSelector getColumnValueSelectorFromDimensionSpec( String dimName = dimSpec.getDimension(); ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(dimName); capabilities = getEffectiveCapabilities(dimSpec, capabilities); - if (capabilities.getType() == ValueType.STRING) { + if (capabilities.is(ValueType.STRING)) { return columnSelectorFactory.makeDimensionSelector(dimSpec); } return columnSelectorFactory.makeColumnValueSelector(dimSpec.getDimension()); @@ -249,7 +252,7 @@ private static ColumnCapabilities getEffectiveCapabilities( } // Complex dimension type is not supported - if (capabilities.getType() == ValueType.COMPLEX) { + if (capabilities.is(ValueType.COMPLEX)) { capabilities = DEFAULT_STRING_CAPABILITIES; } @@ -258,7 +261,7 @@ private static ColumnCapabilities getEffectiveCapabilities( if (dimSpec.getExtractionFn() != null) { ExtractionFn fn = dimSpec.getExtractionFn(); capabilities = ColumnCapabilitiesImpl.copyOf(capabilities) - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryValuesUnique( capabilities.isDictionaryEncoded().isTrue() && fn.getExtractionType() == ExtractionFn.ExtractionType.ONE_TO_ONE @@ -270,7 +273,7 @@ private static ColumnCapabilities getEffectiveCapabilities( // DimensionSpec's decorate only operates on DimensionSelectors, so if a spec mustDecorate(), // we need to wrap selectors on numeric columns with a string casting DimensionSelector. - if (ValueType.isNumeric(capabilities.getType())) { + if (capabilities.isNumeric()) { if (dimSpec.mustDecorate()) { capabilities = DEFAULT_STRING_CAPABILITIES; } @@ -358,13 +361,13 @@ public static Float convertObjectToFloat(@Nullable Object valObj, boolean report @Nullable public static Comparable convertObjectToType( @Nullable final Object obj, - final ValueType type, + final TypeSignature type, final boolean reportParseExceptions ) { Preconditions.checkNotNull(type, "type"); - switch (type) { + switch (type.getType()) { case LONG: return convertObjectToLong(obj, reportParseExceptions); case FLOAT: @@ -381,7 +384,7 @@ public static Comparable convertObjectToType( public static int compareObjectsAsType( @Nullable final Object lhs, @Nullable final Object rhs, - final ValueType type + final ColumnType type ) { //noinspection unchecked @@ -392,17 +395,17 @@ public static int compareObjectsAsType( } @Nullable - public static Comparable convertObjectToType(@Nullable final Object obj, final ValueType type) + public static Comparable convertObjectToType(@Nullable final Object obj, final TypeSignature type) { return convertObjectToType(obj, Preconditions.checkNotNull(type, "type"), false); } public static Function> converterFromTypeToType( - final ValueType fromType, - final ValueType toType + final TypeSignature fromType, + final TypeSignature toType ) { - if (fromType == toType) { + if (Objects.equals(fromType, toType)) { //noinspection unchecked return (Function) Function.identity(); } else { diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java index beb58df6208b..f943f2f163ba 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -27,7 +27,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; @@ -102,7 +102,7 @@ public int getCardinality() @Override public ColumnCapabilities getColumnCapabilities() { - ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE); + ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE); if (hasNulls) { builder.setHasNulls(hasNulls); } diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java index 8d86315d7358..21bd1a65c1b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -27,7 +27,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; @@ -103,7 +103,7 @@ public int getCardinality() @Override public ColumnCapabilities getColumnCapabilities() { - ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT); + ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT); if (hasNulls) { builder.setHasNulls(hasNulls); } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index dd4425b99647..709256ef681a 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -52,6 +52,7 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnDescriptor; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -239,9 +240,9 @@ private static void validateRowValues( ColumnCapabilities capabilities1 = adapter1.getCapabilities(dim1Name); ColumnCapabilities capabilities2 = adapter2.getCapabilities(dim2Name); - ValueType dim1Type = capabilities1.getType(); - ValueType dim2Type = capabilities2.getType(); - if (dim1Type != dim2Type) { + ColumnType dim1Type = capabilities1.toColumnType(); + ColumnType dim2Type = capabilities2.toColumnType(); + if (!Objects.equals(dim1Type, dim2Type)) { throw new SegmentValidationException( "Dim [%s] types not equal. Expected %d found %d", dim1Name, diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java index d9534bf77a5b..bce27c27fcc9 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -27,7 +27,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; @@ -104,7 +104,7 @@ public int getCardinality() @Override public ColumnCapabilities getColumnCapabilities() { - ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); + ColumnCapabilitiesImpl builder = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG); if (hasNulls) { builder.setHasNulls(hasNulls); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java index 2f05a2fa2dc8..bbd9e0c730c4 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexColumnSelectorFactory.java @@ -26,7 +26,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.column.ValueTypes; import org.apache.druid.segment.data.ReadableOffset; @@ -117,9 +116,13 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, descending); } - ValueType type = columnHolder.getCapabilities().getType(); - if (type.isNumeric()) { - return ValueTypes.makeNumericWrappingDimensionSelector(type, makeColumnValueSelector(dimension), extractionFn); + ColumnCapabilities capabilities = columnHolder.getCapabilities(); + if (columnHolder.getCapabilities().isNumeric()) { + return ValueTypes.makeNumericWrappingDimensionSelector( + capabilities.getType(), + makeColumnValueSelector(dimension), + extractionFn + ); } final DictionaryEncodedColumn column = getCachedColumn(dimension, DictionaryEncodedColumn.class); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index e68a119113ff..41a3087c77b4 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -32,7 +32,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ComplexColumn; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.ImmutableBitmapValues; @@ -355,8 +354,7 @@ public String getMetricType(String metric) { final ColumnHolder columnHolder = input.getColumnHolder(metric); - final ValueType type = columnHolder.getCapabilities().getType(); - switch (type) { + switch (columnHolder.getCapabilities().getType()) { case FLOAT: return "float"; case LONG: @@ -369,7 +367,7 @@ public String getMetricType(String metric) } } default: - throw new ISE("Unknown type[%s]", type); + throw new ISE("Unknown type[%s]", columnHolder.getCapabilities().asTypeString()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index 84cd638d09a6..35a8443a35bc 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -186,7 +186,7 @@ public String getColumnTypeName(String columnName) if (col instanceof ComplexColumn) { return ((ComplexColumn) col).getTypeName(); } else { - return columnHolder.getCapabilities().getType().toString(); + return columnHolder.getCapabilities().asTypeString(); } } catch (IOException e) { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java index bfbdc531ec58..ea1a48ed50f6 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java @@ -30,8 +30,8 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.RangeIndexedInts; @@ -96,9 +96,9 @@ static ColumnCapabilities getColumnCapabilities( { if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { // TIME_COLUMN_NAME is handled specially; override the provided rowSignature. - return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG); } else { - final ValueType valueType = rowSignature.getColumnType(columnName).orElse(null); + final ColumnType valueType = rowSignature.getColumnType(columnName).orElse(null); if (valueType != null) { diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java index cfd6aafb8820..d962accfcb7e 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedStorageAdapter.java @@ -125,7 +125,7 @@ public ColumnCapabilities getColumnCapabilities(String column) public String getColumnTypeName(String column) { final ColumnCapabilities columnCapabilities = getColumnCapabilities(column); - return columnCapabilities != null ? columnCapabilities.getType().toString() : null; + return columnCapabilities != null ? columnCapabilities.asTypeString() : null; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 808a8c301fef..f2b4647e92e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -42,7 +42,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.IndexedInts; @@ -480,7 +480,7 @@ public int getUnsortedEncodedKeyComponentHashCode(int[] key) @Override public ColumnCapabilities getColumnCapabilities() { - ColumnCapabilitiesImpl capabilites = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilitiesImpl capabilites = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasBitmapIndexes(hasBitmapIndexes) .setHasSpatialIndexes(hasSpatialIndexes) .setDictionaryValuesUnique(true) diff --git a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java index d3d569747ebb..d9f4bbce456d 100644 --- a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java @@ -108,7 +108,7 @@ T makeMultiValueDimensionProcessor( default boolean useDictionaryEncodedSelector(ColumnCapabilities capabilities) { Preconditions.checkArgument(capabilities != null, "Capabilities must not be null"); - Preconditions.checkArgument(capabilities.getType() == ValueType.STRING, "Must only be called on a STRING column"); + Preconditions.checkArgument(capabilities.is(ValueType.STRING), "Must only be called on a STRING column"); return capabilities.isDictionaryEncoded().and(capabilities.areDictionaryValuesUnique()).isTrue(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java index c7cb8d76e3c8..1af077a266f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java @@ -54,13 +54,13 @@ public SmooshedFileMapper getFileMapper() public ColumnBuilder setType(ValueType type) { - this.capabilitiesBuilder.setType(type); + this.capabilitiesBuilder.setType(ColumnTypeFactory.ofValueType(type)); return this; } public ColumnBuilder setComplexTypeName(String typeName) { - this.capabilitiesBuilder.setComplexTypeName(typeName); + this.capabilitiesBuilder.setType(ColumnType.ofComplex(typeName)); return this; } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java index e6d0920748fe..c6549da02691 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java @@ -29,21 +29,26 @@ * This interface is used to expose information about columns that is interesting to know for all matters dealing with * reading from columns, including query planning and optimization, creating readers to merge segments at ingestion * time, and probably nearly anything else you can imagine. + * + * @see ColumnType + * @see TypeSignature */ -public interface ColumnCapabilities +public interface ColumnCapabilities extends TypeSignature { /** * Column type, good to know so caller can know what to expect and which optimal selector to use */ + @Override ValueType getType(); - /** - * - * If ValueType is COMPLEX, then the typeName associated with it. - */ @Nullable + @Override String getComplexTypeName(); + @Nullable + @Override + TypeSignature getElementType(); + /** * Is the column dictionary encoded? If so, a DimensionDictionarySelector may be used instead of using a value * selector, allowing algorithms to operate on primitive integer dictionary ids rather than the looked up dictionary @@ -95,6 +100,11 @@ public interface ColumnCapabilities */ Capable hasNulls(); + default ColumnType toColumnType() + { + return ColumnTypeFactory.ofType(this); + } + enum Capable { FALSE, diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java index 7b3dd448c5d3..b65bd8e12387 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java @@ -72,6 +72,7 @@ public static ColumnCapabilitiesImpl copyOf(@Nullable final ColumnCapabilities o if (other != null) { capabilities.type = other.getType(); capabilities.complexTypeName = other.getComplexTypeName(); + capabilities.elementType = other.getElementType(); capabilities.dictionaryEncoded = other.isDictionaryEncoded(); capabilities.hasInvertedIndexes = other.hasBitmapIndexes(); capabilities.hasSpatialIndexes = other.hasSpatialIndexes(); @@ -130,14 +131,6 @@ public static ColumnCapabilitiesImpl merge( throw new ISE("Cannot merge columns of type[%s] and [%s]", merged.type, otherSnapshot.getType()); } - if (merged.type == ValueType.COMPLEX && merged.complexTypeName == null) { - merged.complexTypeName = other.getComplexTypeName(); - } - - if (merged.type == ValueType.COMPLEX && merged.complexTypeName != null && !merged.complexTypeName.equals(other.getComplexTypeName())) { - throw new ISE("Cannot merge columns of typeName[%s] and [%s]", merged.complexTypeName, other.getComplexTypeName()); - } - merged.dictionaryEncoded = merged.dictionaryEncoded.or(otherSnapshot.isDictionaryEncoded()); merged.hasMultipleValues = merged.hasMultipleValues.or(otherSnapshot.hasMultipleValues()); merged.dictionaryValuesSorted = merged.dictionaryValuesSorted.and(otherSnapshot.areDictionaryValuesSorted()); @@ -161,8 +154,9 @@ public static ColumnCapabilitiesImpl createDefault() /** * Create a no frills, simple column with {@link ValueType} set and everything else false + * @param valueType */ - public static ColumnCapabilitiesImpl createSimpleNumericColumnCapabilities(ValueType valueType) + public static ColumnCapabilitiesImpl createSimpleNumericColumnCapabilities(ColumnType valueType) { ColumnCapabilitiesImpl builder = new ColumnCapabilitiesImpl().setType(valueType) .setHasMultipleValues(false) @@ -182,7 +176,7 @@ public static ColumnCapabilitiesImpl createSimpleNumericColumnCapabilities(Value */ public static ColumnCapabilitiesImpl createSimpleSingleValueStringColumnCapabilities() { - return new ColumnCapabilitiesImpl().setType(ValueType.STRING) + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(false) .setDictionaryEncoded(false) @@ -195,8 +189,9 @@ public static ColumnCapabilitiesImpl createSimpleSingleValueStringColumnCapabili /** * Similar to {@link #createSimpleNumericColumnCapabilities} except {@link #hasMultipleValues} is explicitly true * and {@link #hasNulls} is not set + * @param valueType */ - public static ColumnCapabilitiesImpl createSimpleArrayColumnCapabilities(ValueType valueType) + public static ColumnCapabilitiesImpl createSimpleArrayColumnCapabilities(ColumnType valueType) { ColumnCapabilitiesImpl builder = new ColumnCapabilitiesImpl().setType(valueType) .setHasMultipleValues(true) @@ -210,9 +205,10 @@ public static ColumnCapabilitiesImpl createSimpleArrayColumnCapabilities(ValueTy @Nullable private ValueType type = null; - @Nullable - private String complexTypeName = null; + private String complexTypeName; + @Nullable + private TypeSignature elementType; private boolean hasInvertedIndexes = false; private boolean hasSpatialIndexes = false; @@ -236,22 +232,26 @@ public ValueType getType() return type; } + @Nullable @Override - @JsonProperty public String getComplexTypeName() { return complexTypeName; } - public ColumnCapabilitiesImpl setType(ValueType type) + @Nullable + @Override + public TypeSignature getElementType() { - this.type = Preconditions.checkNotNull(type, "'type' must be nonnull"); - return this; + return elementType; } - public ColumnCapabilitiesImpl setComplexTypeName(String typeName) + public ColumnCapabilitiesImpl setType(ColumnType type) { - this.complexTypeName = typeName; + Preconditions.checkNotNull(type, "'type' must be nonnull"); + this.type = type.getType(); + this.complexTypeName = type.getComplexTypeName(); + this.elementType = type.getElementType(); return this; } @@ -353,11 +353,7 @@ public ColumnCapabilitiesImpl setHasNulls(Capable hasNulls) @Override public boolean isFilterable() { - return type == ValueType.STRING || - type == ValueType.LONG || - type == ValueType.FLOAT || - type == ValueType.DOUBLE || - filterable; + return (type != null && (isPrimitive() || isArray())) || filterable; } public ColumnCapabilitiesImpl setFilterable(boolean filterable) diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java index fbb5b1d1bbca..0783f7586c11 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.serde.ColumnPartSerde; +import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.serde.Serializer; import javax.annotation.Nullable; @@ -105,6 +106,9 @@ public ColumnHolder read(ByteBuffer buffer, ColumnConfig columnConfig, SmooshedF .setFileMapper(smooshedFiles); for (ColumnPartSerde part : parts) { + if (part instanceof ComplexColumnPartSerde) { + builder.setComplexTypeName(((ComplexColumnPartSerde) part).getTypeName()); + } part.getDeserializer().read(buffer, builder, columnConfig); } diff --git a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java index c988f08acc96..5f88ee4458ef 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java @@ -51,19 +51,19 @@ public class RowSignature implements ColumnInspector { private static final RowSignature EMPTY = new RowSignature(Collections.emptyList()); - private final Map columnTypes = new HashMap<>(); + private final Map columnTypes = new HashMap<>(); private final Object2IntMap columnPositions = new Object2IntOpenHashMap<>(); private final List columnNames; - private RowSignature(final List> columnTypeList) + private RowSignature(final List> columnTypeList) { this.columnPositions.defaultReturnValue(-1); final ImmutableList.Builder columnNamesBuilder = ImmutableList.builder(); for (int i = 0; i < columnTypeList.size(); i++) { - final Pair pair = columnTypeList.get(i); - final ValueType existingType = columnTypes.get(pair.lhs); + final Pair pair = columnTypeList.get(i); + final ColumnType existingType = columnTypes.get(pair.lhs); if (columnTypes.containsKey(pair.lhs) && existingType != pair.rhs) { // It's ok to add the same column twice as long as the type is consistent. @@ -103,7 +103,7 @@ public String getColumnName(final int columnNumber) * Returns the type of the column named {@code columnName}, or empty if the type is unknown or the column does * not exist. */ - public Optional getColumnType(final String columnName) + public Optional getColumnType(final String columnName) { return Optional.ofNullable(columnTypes.get(columnName)); } @@ -113,7 +113,7 @@ public Optional getColumnType(final String columnName) * * @throws IndexOutOfBoundsException if columnNumber is not within our row length */ - public Optional getColumnType(final int columnNumber) + public Optional getColumnType(final int columnNumber) { return Optional.ofNullable(columnTypes.get(getColumnName(columnNumber))); } @@ -206,7 +206,7 @@ public ColumnCapabilities getColumnCapabilities(String column) public static class Builder { - private final List> columnTypeList; + private final List> columnTypeList; private Builder() { @@ -215,11 +215,10 @@ private Builder() /** * Add a column to this signature. - * - * @param columnName name, must be nonnull + * @param columnName name, must be nonnull * @param columnType type, may be null if unknown */ - public Builder add(final String columnName, @Nullable final ValueType columnType) + public Builder add(final String columnName, @Nullable final ColumnType columnType) { // Name must be nonnull, but type can be null (if the type is unknown) Preconditions.checkNotNull(columnName, "'columnName' must be non-null"); @@ -238,7 +237,7 @@ public Builder addAll(final RowSignature other) public Builder addTimeColumn() { - return add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG); + return add(ColumnHolder.TIME_COLUMN_NAME, ColumnType.LONG); } public Builder addDimensions(final List dimensions) @@ -253,7 +252,7 @@ public Builder addDimensions(final List dimensions) public Builder addAggregators(final List aggregators) { for (final AggregatorFactory aggregator : aggregators) { - final ValueType type = aggregator.getType(); + final ColumnType type = aggregator.getType(); if (type.equals(aggregator.getFinalizedType())) { add(aggregator.getName(), type); diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java index 32bb43078c36..72e623159073 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java @@ -58,7 +58,7 @@ class SimpleColumnHolder implements ColumnHolder // are prone to such backward incompatible changes. if (columnSupplier == null) { Preconditions.checkArgument( - capabilities.getType() == ValueType.COMPLEX, + capabilities.is(ValueType.COMPLEX), "Only complex column types can have nullable column suppliers" ); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java index e636572e399a..3042325e6a44 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java @@ -35,7 +35,7 @@ import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import javax.annotation.Nullable; @@ -189,9 +189,9 @@ private static class ColumnComparisonReaderFactory implements ColumnProcessorFac private static final String[] NULL_VALUE = new String[]{null}; @Override - public ValueType defaultType() + public ColumnType defaultType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java index eeffa44f7757..7dae76cf5422 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java @@ -28,7 +28,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; @@ -46,9 +46,10 @@ public class ConstantValueMatcherFactory implements ColumnProcessorFactory makeColumnValueSelector(final String column) { - final boolean isComplexMetric = ValueType.COMPLEX.equals(agg.getType()); + final boolean isComplexMetric = agg.getType().is(ValueType.COMPLEX); final ColumnValueSelector selector = baseSelectorFactory.makeColumnValueSelector(column); @@ -137,7 +138,7 @@ public ColumnValueSelector makeColumnValueSelector(final String column) // Wrap selector in a special one that uses ComplexMetricSerde to modify incoming objects. // For complex aggregators that read from multiple columns, we wrap all of them. This is not ideal but it // has worked so far. - final String complexTypeName = agg.getComplexTypeName(); + final String complexTypeName = agg.getType().getComplexTypeName(); final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(complexTypeName); if (serde == null) { throw new ISE("Don't know how to handle type[%s]", complexTypeName); @@ -287,13 +288,13 @@ protected IncrementalIndex( this.dimensionDescsList = new ArrayList<>(); for (DimensionSchema dimSchema : dimensionsSpec.getDimensions()) { - ValueType type = dimSchema.getValueType(); + ColumnType type = dimSchema.getColumnType(); String dimName = dimSchema.getName(); // Note: Things might be simpler if DimensionSchema had a method "getColumnCapabilities()" which could return // type specific capabilities by itself. However, for various reasons, DimensionSchema currently lives in druid-core // while ColumnCapabilities lives in druid-processing which makes that approach difficult. - ColumnCapabilitiesImpl capabilities = makeDefaultCapabilitiesFromValueType(type, dimSchema.getTypeName()); + ColumnCapabilitiesImpl capabilities = makeDefaultCapabilitiesFromValueType(type); capabilities.setHasBitmapIndexes(dimSchema.hasBitmapIndex()); @@ -311,7 +312,7 @@ protected IncrementalIndex( //__time capabilities timeAndMetricsColumnCapabilities.put( ColumnHolder.TIME_COLUMN_NAME, - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG) ); // This should really be more generic @@ -533,7 +534,7 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) dimension, // for schemaless type discovery, everything is a String. this should probably try to autodetect // based on the value to use a better handler - makeDefaultCapabilitiesFromValueType(ValueType.STRING, null), + makeDefaultCapabilitiesFromValueType(ColumnType.STRING), null ) ); @@ -784,9 +785,9 @@ public List getDimensionOrder() } } - private ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ValueType type, @Nullable String typeName) + private ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ColumnType type) { - switch (type) { + switch (type.getType()) { case STRING: // we start out as not having multiple values, but this might change as we encounter them return new ColumnCapabilitiesImpl().setType(type) @@ -795,7 +796,7 @@ private ColumnCapabilitiesImpl makeDefaultCapabilitiesFromValueType(ValueType ty .setDictionaryValuesUnique(true) .setDictionaryValuesSorted(false); case COMPLEX: - return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(type).setHasNulls(true).setComplexTypeName(typeName); + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(type).setHasNulls(true); default: return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(type); } @@ -993,20 +994,19 @@ public MetricDesc(int index, AggregatorFactory factory) this.index = index; this.name = factory.getName(); - ValueType valueType = factory.getType(); + ColumnType valueType = factory.getType(); if (valueType.isNumeric()) { capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType); this.type = valueType.toString(); - } else if (ValueType.COMPLEX.equals(valueType)) { - capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.COMPLEX) + } else if (valueType.is(ValueType.COMPLEX)) { + capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType) .setHasNulls(ColumnCapabilities.Capable.TRUE); - String complexTypeName = factory.getComplexTypeName(); - ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(complexTypeName); + ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(valueType.getComplexTypeName()); if (serde != null) { this.type = serde.getTypeName(); } else { - throw new ISE("Unable to handle complex type[%s] of type[%s]", complexTypeName, valueType); + throw new ISE("Unable to handle complex type[%s]", valueType); } } else { // if we need to handle non-numeric and non-complex types (e.g. strings, arrays) it should be done here diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index 6ecb83f47a9e..7f0b1dc0e9bc 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -85,7 +85,7 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi if (capabilities == null) { return DimensionSelector.constant(null, extractionFn); } - if (capabilities.getType().isNumeric()) { + if (capabilities.isNumeric()) { return ValueTypes.makeNumericWrappingDimensionSelector( capabilities.getType(), makeColumnValueSelector(dimension), diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index e30879883385..0b8047eb19c7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -248,7 +248,7 @@ public String getColumnTypeName(String column) } ColumnCapabilities columnCapabilities = getColumnCapabilities(column); if (columnCapabilities != null) { - return columnCapabilities.getType().toString(); + return columnCapabilities.asTypeString(); } else { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index c056725cad99..f27a279fbb75 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -197,7 +197,7 @@ public String getColumnTypeName(String column) if (maybeClause.isPresent()) { final JoinableClause clause = maybeClause.get(); final ColumnCapabilities capabilities = clause.getJoinable().getColumnCapabilities(clause.unprefix(column)); - return capabilities != null ? capabilities.getType().toString() : null; + return capabilities != null ? capabilities.asTypeString() : null; } else { return baseAdapter.getColumnTypeName(column); } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java index 1fb9cc61d174..2420369d9e8d 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java @@ -26,7 +26,7 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.filter.FalseFilter; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.OrFilter; @@ -302,7 +302,7 @@ private static JoinFilterAnalysis rewriteFilterDirect( return new ExpressionVirtualColumn( vcName, correlatedBaseExpr, - ValueType.STRING + ColumnType.STRING ); } ); @@ -466,7 +466,7 @@ private static JoinFilterAnalysis rewriteSelectorFilter( return new ExpressionVirtualColumn( vcName, correlatedBaseExpr, - ValueType.STRING + ColumnType.STRING ); } ); diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java index 845b5cb45623..912278970096 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupColumnSelectorFactory.java @@ -29,7 +29,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -105,7 +105,7 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) public ColumnCapabilities getColumnCapabilities(String column) { if (LookupJoinable.ALL_COLUMNS.contains(column)) { - return new ColumnCapabilitiesImpl().setType(ValueType.STRING); + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING); } else { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java index a97739b632e2..c1703640330f 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java @@ -37,7 +37,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.join.Equality; import org.apache.druid.segment.join.JoinConditionAnalysis; @@ -59,9 +59,9 @@ public class LookupJoinMatcher implements JoinMatcher new ColumnProcessorFactory>() { @Override - public ValueType defaultType() + public ColumnType defaultType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override @@ -155,7 +155,7 @@ private LookupJoinMatcher( expr -> ColumnProcessors.makeProcessor( expr, - ValueType.STRING, + ColumnType.STRING, LEFT_KEY_READER, leftSelectorFactory ) diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java index 2d3c43d76883..d2b399119f40 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java @@ -28,7 +28,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinMatcher; import org.apache.druid.segment.join.Joinable; @@ -77,7 +77,7 @@ public int getCardinality(String columnName) public ColumnCapabilities getColumnCapabilities(String columnName) { if (ALL_COLUMNS.contains(columnName)) { - return new ColumnCapabilitiesImpl().setType(ValueType.STRING); + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING); } else { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java index 5de8fe0a4bc2..f143f1d24a22 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/BroadcastSegmentIndexedTable.java @@ -39,8 +39,8 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnHolder; +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.ReadableOffset; import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; @@ -90,9 +90,9 @@ public BroadcastSegmentIndexedTable( ); RowSignature.Builder sigBuilder = RowSignature.builder(); - sigBuilder.add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG); + sigBuilder.add(ColumnHolder.TIME_COLUMN_NAME, ColumnType.LONG); for (String column : queryableIndex.getColumnNames()) { - sigBuilder.add(column, adapter.getColumnCapabilities(column).getType()); + sigBuilder.add(column, adapter.getColumnCapabilities(column).toColumnType()); } this.rowSignature = sigBuilder.build(); @@ -103,7 +103,7 @@ public BroadcastSegmentIndexedTable( final RowBasedIndexBuilder m; final String columnName = rowSignature.getColumnName(i); if (keyColumns.contains(columnName)) { - final ValueType keyType = + final ColumnType keyType = rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE); m = new RowBasedIndexBuilder(keyType); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java index 05dd92ad8c82..d1de1ffd3ff8 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ReferenceCountedObject; +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.ReadableOffset; @@ -119,7 +120,7 @@ interface Index /** * Returns the natural key type for the index. */ - ValueType keyType(); + ColumnType keyType(); /** * Returns whether keys are unique in this index. If this returns true, then {@link #find(Object)} will only ever diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java index 26fe2a9c1c5e..97a9fdd3d595 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java @@ -27,6 +27,7 @@ import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import javax.annotation.Nonnull; @@ -49,12 +50,12 @@ public class IndexedTableColumnSelectorFactory implements ColumnSelectorFactory @Nullable static ColumnCapabilities columnCapabilities(final IndexedTable table, final String columnName) { - final ValueType valueType = table.rowSignature().getColumnType(columnName).orElse(null); + final ColumnType valueType = table.rowSignature().getColumnType(columnName).orElse(null); if (valueType != null) { final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(valueType); - if (valueType == ValueType.STRING) { + if (valueType.is(ValueType.STRING)) { // IndexedTables are not _really_ dictionary-encoded, but we fake it using the row number as the dict. code. capabilities.setDictionaryEncoded(true); } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java index 89a1ea9c0332..87af991f0a21 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java @@ -45,6 +45,7 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.join.Equality; @@ -68,7 +69,7 @@ public class IndexedTableJoinMatcher implements JoinMatcher private static final int UNINITIALIZED_CURRENT_ROW = -1; // Key column type to use when the actual key type is unknown. - static final ValueType DEFAULT_KEY_TYPE = ValueType.STRING; + static final ColumnType DEFAULT_KEY_TYPE = ColumnType.STRING; private final IndexedTable table; private final List conditionMatchers; @@ -343,7 +344,7 @@ static class ConditionMatcherFactory implements ColumnProcessorFactory (int) dimension id -> (IntList) row numbers @@ -379,7 +380,7 @@ private IntList getAndCacheRowNumbers(DimensionSelector selector, int dimensionI } @Override - public ValueType defaultType() + public ColumnType defaultType() { return keyType; } @@ -456,7 +457,7 @@ public ConditionMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector select @Override public ConditionMatcher makeLongProcessor(BaseLongColumnValueSelector selector) { - if (index.keyType() == ValueType.LONG) { + if (index.keyType().is(ValueType.LONG)) { return makePrimitiveLongMatcher(selector); } else if (NullHandling.replaceWithDefault()) { return () -> index.find(selector.getLong()).iterator(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/MapIndex.java b/processing/src/main/java/org/apache/druid/segment/join/table/MapIndex.java index df93f1b1d476..30d6ec04da7d 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/MapIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/MapIndex.java @@ -24,7 +24,7 @@ import it.unimi.dsi.fastutil.ints.IntLists; import it.unimi.dsi.fastutil.longs.Long2ObjectMap; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import java.util.Map; @@ -33,7 +33,7 @@ */ public class MapIndex implements IndexedTable.Index { - private final ValueType keyType; + private final ColumnType keyType; private final Map index; private final boolean keysUnique; private final boolean isLong2ObjectMap; @@ -47,7 +47,7 @@ public class MapIndex implements IndexedTable.Index * * @see RowBasedIndexBuilder#build() the main caller */ - MapIndex(final ValueType keyType, final Map index, final boolean keysUnique) + MapIndex(final ColumnType keyType, final Map index, final boolean keysUnique) { this.keyType = Preconditions.checkNotNull(keyType, "keyType"); this.index = Preconditions.checkNotNull(index, "index"); @@ -56,7 +56,7 @@ public class MapIndex implements IndexedTable.Index } @Override - public ValueType keyType() + public ColumnType keyType() { return keyType; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexBuilder.java b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexBuilder.java index 605edb950955..edb89633351a 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexBuilder.java @@ -27,6 +27,7 @@ import it.unimi.dsi.fastutil.objects.ObjectIterator; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import javax.annotation.Nullable; @@ -49,17 +50,17 @@ public class RowBasedIndexBuilder private int currentRow = 0; private int nullKeys = 0; - private final ValueType keyType; + private final ColumnType keyType; private final Map index; private long minLongKey = Long.MAX_VALUE; private long maxLongKey = Long.MIN_VALUE; - public RowBasedIndexBuilder(ValueType keyType) + public RowBasedIndexBuilder(ColumnType keyType) { this.keyType = keyType; - if (keyType == ValueType.LONG) { + if (keyType.is(ValueType.LONG)) { // We're specializing the type even though we don't specialize usage in this class, for two reasons: // (1) It's still useful to reduce overall memory footprint. // (2) MapIndex specifically checks for Long2ObjectMap instances and *does* specialize usage. @@ -84,11 +85,11 @@ public RowBasedIndexBuilder add(@Nullable final Object key) rowNums.add(currentRow); // Track min, max long value so we can decide later on if it's appropriate to use an array-backed implementation. - if (keyType == ValueType.LONG && (long) castKey < minLongKey) { + if (keyType.is(ValueType.LONG) && (long) castKey < minLongKey) { minLongKey = (long) castKey; } - if (keyType == ValueType.LONG && (long) castKey > maxLongKey) { + if (keyType.is(ValueType.LONG) && (long) castKey > maxLongKey) { maxLongKey = (long) castKey; } } else { @@ -107,7 +108,7 @@ public IndexedTable.Index build() { final boolean keysUnique = index.size() == currentRow - nullKeys; - if (keyType == ValueType.LONG && keysUnique && index.size() > 0) { + if (keyType.is(ValueType.LONG) && keysUnique && index.size() > 0) { // May be a good candidate for UniqueLongArrayIndex. Check the range of values as compared to min and max. long range; diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java index c21ad195a57c..8945f50e44f7 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java @@ -24,8 +24,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import javax.annotation.Nullable; import java.io.Closeable; @@ -102,7 +102,7 @@ public RowBasedIndexedTable( final Index m; if (keyColumns.contains(column)) { - final ValueType keyType = + final ColumnType keyType = rowSignature.getColumnType(column).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE); final RowBasedIndexBuilder builder = new RowBasedIndexBuilder(keyType); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/UniqueLongArrayIndex.java b/processing/src/main/java/org/apache/druid/segment/join/table/UniqueLongArrayIndex.java index 5339b6ea1cfe..09dc02ce72a5 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/UniqueLongArrayIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/UniqueLongArrayIndex.java @@ -22,7 +22,7 @@ import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.ints.IntLists; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; /** * An {@link IndexedTable.Index} backed by an int array. @@ -49,9 +49,9 @@ public class UniqueLongArrayIndex implements IndexedTable.Index } @Override - public ValueType keyType() + public ColumnType keyType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java index 40d2f950dcff..af9e2832cb08 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java @@ -99,7 +99,7 @@ public MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(final D final ColumnHolder holder = index.getColumnHolder(spec.getDimension()); if (holder == null || holder.getCapabilities().isDictionaryEncoded().isFalse() - || holder.getCapabilities().getType() != ValueType.STRING + || !holder.getCapabilities().is(ValueType.STRING) || holder.getCapabilities().hasMultipleValues().isFalse()) { throw new ISE( "Column[%s] is not a multi-value string column, do not ask for a multi-value selector", @@ -155,7 +155,7 @@ public SingleValueDimensionVectorSelector makeSingleValueDimensionSelector(final final ColumnHolder holder = index.getColumnHolder(spec.getDimension()); if (holder == null || !holder.getCapabilities().isDictionaryEncoded().isTrue() - || holder.getCapabilities().getType() != ValueType.STRING) { + || !holder.getCapabilities().is(ValueType.STRING)) { // Asking for a single-value dimension selector on a non-string column gets you a bunch of nulls. return NilVectorSelector.create(offset); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlan.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlan.java index 59ef0349b346..fc7c14c319ea 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlan.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlan.java @@ -22,11 +22,13 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import javax.annotation.Nullable; @@ -87,9 +89,9 @@ public enum Trait private final EnumSet traits; @Nullable - private final ExprType outputType; + private final ExpressionType outputType; @Nullable - private final ValueType singleInputType; + private final ColumnType singleInputType; private final Set unknownInputs; private final List unappliedInputs; @@ -98,8 +100,8 @@ public enum Trait Expr expression, Expr.BindingAnalysis analysis, EnumSet traits, - @Nullable ExprType outputType, - @Nullable ValueType singleInputType, + @Nullable ExpressionType outputType, + @Nullable ColumnType singleInputType, Set unknownInputs, List unappliedInputs ) @@ -171,7 +173,7 @@ public Expr getAppliedFoldExpression(String accumulatorId) * or {@link #getAppliedFoldExpression(String)}, should the expression have any unapplied inputs */ @Nullable - public ExprType getOutputType() + public ExpressionType getOutputType() { return outputType; } @@ -180,7 +182,7 @@ public ExprType getOutputType() * If and only if the column has a single input, get the {@link ValueType} of that input */ @Nullable - public ValueType getSingleInputType() + public ColumnType getSingleInputType() { return singleInputType; } @@ -219,16 +221,16 @@ public Expr.BindingAnalysis getAnalysis() * If no output type was able to be inferred during planning, returns null */ @Nullable - public ColumnCapabilities inferColumnCapabilities(@Nullable ValueType outputTypeHint) + public ColumnCapabilities inferColumnCapabilities(@Nullable ColumnType outputTypeHint) { if (outputType != null) { - final ValueType inferredValueType = ExprType.toValueType(outputType); + final ColumnType inferredValueType = ExpressionType.toColumnType(outputType); if (inferredValueType.isNumeric()) { // if float was explicitly specified preserve it, because it will currently never be the computed output type // since there is no float expression type - if (ValueType.FLOAT == outputTypeHint) { - return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT); + if (Types.is(outputTypeHint, ValueType.FLOAT)) { + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT); } return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(inferredValueType); } @@ -240,7 +242,7 @@ public ColumnCapabilities inferColumnCapabilities(@Nullable ValueType outputType } // fancy string stuffs - if (ValueType.STRING == inferredValueType) { + if (inferredValueType.is(ValueType.STRING)) { // constant strings are supported as dimension selectors, set them as dictionary encoded and unique for all the // bells and whistles the engines have to offer if (isConstant()) { @@ -261,7 +263,7 @@ public ColumnCapabilities inferColumnCapabilities(@Nullable ValueType outputType // to create a dictionary encoded selector instead of an object selector to defer expression evaluation // until query time return ColumnCapabilitiesImpl.copyOf(underlyingCapabilities) - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryValuesSorted(false) .setDictionaryValuesUnique(false) .setHasNulls(true); @@ -273,11 +275,11 @@ public ColumnCapabilities inferColumnCapabilities(@Nullable ValueType outputType // the complete set of input types if (any(Trait.NON_SCALAR_OUTPUT, Trait.NEEDS_APPLIED)) { // if the hint requested a string, use a string - if (ValueType.STRING == outputTypeHint) { - return ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.STRING); + if (Types.is(outputTypeHint, ValueType.STRING)) { + return ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.STRING); } // maybe something is looking for a little fun and wants arrays? let whatever it is through - return ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ExprType.toValueType(outputType)); + return ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ExpressionType.toColumnType(outputType)); } // if we got here, lets call it single value string output, non-dictionary encoded diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlanner.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlanner.java index 4eb91eb7ae1d..44210be3eed4 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlanner.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionPlanner.java @@ -23,10 +23,11 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import java.util.EnumSet; @@ -60,8 +61,8 @@ public static ExpressionPlan plan(ColumnInspector inspector, Expr expression) Set noCapabilities = new HashSet<>(); Set maybeMultiValued = new HashSet<>(); List needsApplied = ImmutableList.of(); - ValueType singleInputType = null; - ExprType outputType = null; + ColumnType singleInputType = null; + ExpressionType outputType = null; final Set columns = analysis.getRequiredBindings(); @@ -83,7 +84,7 @@ public static ExpressionPlan plan(ColumnInspector inspector, Expr expression) if (capabilities != null && !analysis.hasInputArrays() && !analysis.isOutputArray()) { boolean isSingleInputMappable = false; boolean isSingleInputScalar = capabilities.hasMultipleValues().isFalse(); - if (capabilities.getType() == ValueType.STRING) { + if (capabilities.is(ValueType.STRING)) { isSingleInputScalar &= capabilities.isDictionaryEncoded().isTrue(); isSingleInputMappable = capabilities.isDictionaryEncoded().isTrue() && !capabilities.hasMultipleValues().isUnknown(); @@ -91,7 +92,7 @@ public static ExpressionPlan plan(ColumnInspector inspector, Expr expression) // if satisfied, set single input output type and flags if (isSingleInputScalar || isSingleInputMappable) { - singleInputType = capabilities.getType(); + singleInputType = capabilities.toColumnType(); if (isSingleInputScalar) { traits.add(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR); } @@ -111,7 +112,7 @@ public static ExpressionPlan plan(ColumnInspector inspector, Expr expression) if (capabilities != null) { if (capabilities.hasMultipleValues().isTrue()) { definitelyMultiValued.add(column); - } else if (capabilities.getType().equals(ValueType.STRING) && + } else if (capabilities.is(ValueType.STRING) && capabilities.hasMultipleValues().isMaybeTrue() && !analysis.getArrayBindings().contains(column) ) { @@ -159,7 +160,7 @@ public static ExpressionPlan plan(ColumnInspector inspector, Expr expression) } // if analysis predicts output, or inferred output type is array, output will be multi-valued - if (analysis.isOutputArray() || ExprType.isArray(outputType)) { + if (analysis.isOutputArray() || (outputType != null && outputType.isArray())) { traits.add(ExpressionPlan.Trait.NON_SCALAR_OUTPUT); // single input mappable may not produce array output explicitly, only through implicit mapping diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index d32d6895a828..889ec661964f 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; @@ -141,16 +142,16 @@ public static ColumnValueSelector makeExprEvalSelector( { if (plan.is(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR)) { final String column = plan.getSingleInputName(); - final ValueType inputType = plan.getSingleInputType(); - if (inputType == ValueType.LONG) { + final ColumnType inputType = plan.getSingleInputType(); + if (inputType.is(ValueType.LONG)) { return new SingleLongInputCachingExpressionColumnValueSelector( columnSelectorFactory.makeColumnValueSelector(column), plan.getExpression(), !ColumnHolder.TIME_COLUMN_NAME.equals(column) // __time doesn't need an LRU cache since it is sorted. ); - } else if (inputType == ValueType.STRING) { + } else if (inputType.is(ValueType.STRING)) { return new SingleStringInputCachingExpressionColumnValueSelector( - columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ValueType.STRING)), + columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(column, column, ColumnType.STRING)), plan.getExpression() ); } @@ -186,7 +187,7 @@ public static DimensionSelector makeDimensionSelector( if (plan.any(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR, ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE)) { final String column = plan.getSingleInputName(); - if (plan.getSingleInputType() == ValueType.STRING) { + if (plan.getSingleInputType().is(ValueType.STRING)) { return new SingleStringInputDeferredEvaluationExpressionDimensionSelector( columnSelectorFactory.makeDimensionSelector(DefaultDimensionSpec.of(column)), expression @@ -263,27 +264,26 @@ public static Expr.ObjectBinding createBindings( final Map> suppliers = new HashMap<>(); for (String columnName : columns) { final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(columnName); - final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null; final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues().isTrue(); final Supplier supplier; - if (nativeType == ValueType.FLOAT) { + if (columnCapabilities == null || columnCapabilities.isArray()) { + // Unknown ValueType or array type. Try making an Object selector and see if that gives us anything useful. + supplier = supplierFromObjectSelector(columnSelectorFactory.makeColumnValueSelector(columnName)); + } else if (columnCapabilities.is(ValueType.FLOAT)) { ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(columnName); supplier = makeNullableNumericSupplier(selector, selector::getFloat); - } else if (nativeType == ValueType.LONG) { + } else if (columnCapabilities.is(ValueType.LONG)) { ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(columnName); supplier = makeNullableNumericSupplier(selector, selector::getLong); - } else if (nativeType == ValueType.DOUBLE) { + } else if (columnCapabilities.is(ValueType.DOUBLE)) { ColumnValueSelector selector = columnSelectorFactory.makeColumnValueSelector(columnName); supplier = makeNullableNumericSupplier(selector, selector::getDouble); - } else if (nativeType == ValueType.STRING) { + } else if (columnCapabilities.is(ValueType.STRING)) { supplier = supplierFromDimensionSelector( columnSelectorFactory.makeDimensionSelector(new DefaultDimensionSpec(columnName, columnName)), multiVal ); - } else if (nativeType == null || ValueType.isArray(nativeType)) { - // Unknown ValueType or array type. Try making an Object selector and see if that gives us anything useful. - supplier = supplierFromObjectSelector(columnSelectorFactory.makeColumnValueSelector(columnName)); } else { // Unhandleable ValueType (COMPLEX). supplier = null; @@ -411,15 +411,18 @@ static Supplier supplierFromObjectSelector(final BaseObjectColumnValueSe @Nullable public static Object coerceEvalToSelectorObject(ExprEval eval) { - switch (eval.type()) { - case STRING_ARRAY: - return Arrays.stream(eval.asStringArray()).collect(Collectors.toList()); - case DOUBLE_ARRAY: - return Arrays.stream(eval.asDoubleArray()).collect(Collectors.toList()); - case LONG_ARRAY: - return Arrays.stream(eval.asLongArray()).collect(Collectors.toList()); - default: - return eval.value(); + if (eval.type().isArray()) { + switch (eval.type().getElementType().getType()) { + case STRING: + return Arrays.stream(eval.asStringArray()).collect(Collectors.toList()); + case DOUBLE: + return Arrays.stream(eval.asDoubleArray()).collect(Collectors.toList()); + case LONG: + return Arrays.stream(eval.asLongArray()).collect(Collectors.toList()); + default: + + } } + return eval.value(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorInputBinding.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorInputBinding.java index 93da652e5555..5820c537eab1 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorInputBinding.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorInputBinding.java @@ -20,7 +20,7 @@ package org.apache.druid.segment.virtual; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.segment.vector.NilVectorSelector; import org.apache.druid.segment.vector.ReadableVectorInspector; import org.apache.druid.segment.vector.VectorObjectSelector; @@ -34,7 +34,7 @@ class ExpressionVectorInputBinding implements Expr.VectorInputBinding { private final Map numeric; private final Map objects; - private final Map types; + private final Map types; private final NilVectorSelector nilSelector; private final ReadableVectorInspector vectorInspector; @@ -48,14 +48,14 @@ public ExpressionVectorInputBinding(ReadableVectorInspector vectorInspector) this.nilSelector = NilVectorSelector.create(this.vectorInspector); } - public ExpressionVectorInputBinding addNumeric(String name, ExprType type, VectorValueSelector selector) + public ExpressionVectorInputBinding addNumeric(String name, ExpressionType type, VectorValueSelector selector) { numeric.put(name, selector); types.put(name, type); return this; } - public ExpressionVectorInputBinding addObjectSelector(String name, ExprType type, VectorObjectSelector selector) + public ExpressionVectorInputBinding addObjectSelector(String name, ExpressionType type, VectorObjectSelector selector) { objects.put(name, selector); types.put(name, type); @@ -63,7 +63,7 @@ public ExpressionVectorInputBinding addObjectSelector(String name, ExprType type } @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { return types.get(name); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorSelectors.java index 95e1f6c3eb85..0e2e3cd3b154 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVectorSelectors.java @@ -22,11 +22,11 @@ import com.google.common.base.Preconditions; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.ExprUtils; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.ConstantVectorSelectors; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -55,7 +55,7 @@ public static SingleValueDimensionVectorSelector makeSingleValueDimensionVectorS String constant = plan.getExpression().eval(ExprUtils.nilBindings()).asString(); return ConstantVectorSelectors.singleValueDimensionVectorSelector(factory.getReadableVectorInspector(), constant); } - if (plan.is(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR) && ExprType.STRING == plan.getOutputType()) { + if (plan.is(ExpressionPlan.Trait.SINGLE_INPUT_SCALAR) && (plan.getOutputType() != null && plan.getOutputType().is(ExprType.STRING))) { return new SingleStringInputDeferredEvaluationExpressionDimensionVectorSelector( factory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of(plan.getSingleInputName())), plan.getExpression() @@ -114,22 +114,21 @@ private static Expr.VectorInputBinding createVectorBindings( final List columns = bindingAnalysis.getRequiredBindingsList(); for (String columnName : columns) { final ColumnCapabilities columnCapabilities = vectorColumnSelectorFactory.getColumnCapabilities(columnName); - final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null; // null capabilities should be backed by a nil vector selector since it means the column effectively doesnt exist - if (nativeType != null) { - switch (nativeType) { + if (columnCapabilities != null) { + switch (columnCapabilities.getType()) { case FLOAT: case DOUBLE: - binding.addNumeric(columnName, ExprType.DOUBLE, vectorColumnSelectorFactory.makeValueSelector(columnName)); + binding.addNumeric(columnName, ExpressionType.DOUBLE, vectorColumnSelectorFactory.makeValueSelector(columnName)); break; case LONG: - binding.addNumeric(columnName, ExprType.LONG, vectorColumnSelectorFactory.makeValueSelector(columnName)); + binding.addNumeric(columnName, ExpressionType.LONG, vectorColumnSelectorFactory.makeValueSelector(columnName)); break; default: binding.addObjectSelector( columnName, - ExprType.STRING, + ExpressionType.STRING, vectorColumnSelectorFactory.makeObjectSelector(columnName) ); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java index c0a5d41758f9..d567ca891705 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java @@ -40,7 +40,7 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorObjectSelector; @@ -57,7 +57,7 @@ public class ExpressionVirtualColumn implements VirtualColumn private final String name; private final String expression; @Nullable - private final ValueType outputType; + private final ColumnType outputType; private final Supplier parsedExpression; private final Supplier cacheKey; @@ -65,7 +65,7 @@ public class ExpressionVirtualColumn implements VirtualColumn public ExpressionVirtualColumn( @JsonProperty("name") String name, @JsonProperty("expression") String expression, - @JsonProperty("outputType") @Nullable ValueType outputType, + @JsonProperty("outputType") @Nullable ColumnType outputType, @JacksonInject ExprMacroTable macroTable ) { @@ -82,7 +82,7 @@ public ExpressionVirtualColumn( public ExpressionVirtualColumn( String name, Expr parsedExpression, - ValueType outputType + ColumnType outputType ) { this.name = Preconditions.checkNotNull(name, "name"); @@ -109,7 +109,7 @@ public String getExpression() @Nullable @JsonProperty - public ValueType getOutputType() + public ColumnType getOutputType() { return outputType; } @@ -180,10 +180,10 @@ public ColumnCapabilities capabilities(String columnName) // array types must not currently escape from the expression system if (outputType != null && outputType.isArray()) { - return new ColumnCapabilitiesImpl().setType(ValueType.STRING).setHasMultipleValues(true); + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING).setHasMultipleValues(true); } - return new ColumnCapabilitiesImpl().setType(outputType == null ? ValueType.FLOAT : outputType); + return new ColumnCapabilitiesImpl().setType(outputType == null ? ColumnType.FLOAT : outputType); } @Override @@ -194,20 +194,20 @@ public ColumnCapabilities capabilities(ColumnInspector inspector, String columnN // if we can infer the column capabilities from the expression plan, then use that if (inferred != null) { // explicit outputType is used as a hint, how did it compare to the planners inferred output type? - if (inferred.getType() != outputType && outputType != null) { + if (outputType != null && inferred.getType() != outputType.getType()) { // if both sides are numeric, let it slide and log at debug level // but mismatches involving strings and arrays might be worth knowing about so warn - if (!inferred.getType().isNumeric() && !outputType.isNumeric()) { + if (!inferred.isNumeric() && !outputType.isNumeric()) { log.warn( "Projected output type %s of expression %s does not match provided type %s", - inferred.getType(), + inferred.asTypeString(), expression, outputType ); } else { log.debug( "Projected output type %s of expression %s does not match provided type %s", - inferred.getType(), + inferred.asTypeString(), expression, outputType ); @@ -250,7 +250,7 @@ public boolean equals(final Object o) final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o; return Objects.equals(name, that.name) && Objects.equals(expression, that.expression) && - outputType == that.outputType; + Objects.equals(outputType, that.outputType); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionVectorSelector.java b/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionVectorSelector.java index 56813a491f50..1cee307adb77 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionVectorSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionVectorSelector.java @@ -21,7 +21,7 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.math.expr.Expr; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.vector.ExprVectorProcessor; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.IdLookup; @@ -122,9 +122,9 @@ private static final class StringLookupVectorInputBindings implements Expr.Vecto @Nullable @Override - public ExprType getType(String name) + public ExpressionType getType(String name) { - return ExprType.STRING; + return ExpressionType.STRING; } @Override diff --git a/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java index d514971ac417..eacfbee1bf42 100644 --- a/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/InlineDataSourceTest.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.junit.Assert; @@ -66,12 +67,13 @@ public class InlineDataSourceTest "double_array" ); - private final List expectedColumnTypes = ImmutableList.of( - ValueType.LONG, - ValueType.STRING, - ValueType.DOUBLE, - ValueType.COMPLEX, - ValueType.DOUBLE_ARRAY + private final ColumnType someComplex = new ColumnType(ValueType.COMPLEX, "foo", null); + private final List expectedColumnTypes = ImmutableList.of( + ColumnType.LONG, + ColumnType.STRING, + ColumnType.DOUBLE, + someComplex, + ColumnType.DOUBLE_ARRAY ); private final RowSignature expectedRowSignature; @@ -126,11 +128,11 @@ public void test_getRowSignature() { Assert.assertEquals( RowSignature.builder() - .add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG) - .add("str", ValueType.STRING) - .add("double", ValueType.DOUBLE) - .add("complex", ValueType.COMPLEX) - .add("double_array", ValueType.DOUBLE_ARRAY) + .add(ColumnHolder.TIME_COLUMN_NAME, ColumnType.LONG) + .add("str", ColumnType.STRING) + .add("double", ColumnType.DOUBLE) + .add("complex", someComplex) + .add("double_array", ColumnType.DOUBLE_ARRAY) .build(), listDataSource.getRowSignature() ); diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index be6b2072fb1a..ba3eafab48bf 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -61,7 +61,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -409,7 +409,7 @@ public void testGroupByExpression() new ExpressionVirtualColumn( "texpr", "map(x -> concat(x, 'foo'), tags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -456,7 +456,7 @@ public void testGroupByExpressionMultiMulti() new ExpressionVirtualColumn( "texpr", "cartesian_map((x,y) -> concat(x, y), tags, othertags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -501,7 +501,7 @@ public void testGroupByExpressionMultiMultiAuto() new ExpressionVirtualColumn( "texpr", "map((x) -> concat(x, othertags), tags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -546,7 +546,7 @@ public void testGroupByExpressionMultiMultiAutoAuto() new ExpressionVirtualColumn( "texpr", "concat(tags, othertags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -587,7 +587,7 @@ public void testGroupByExpressionMultiMultiAutoAutoDupeIdentifier() new ExpressionVirtualColumn( "texpr", "concat(tags, tags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -639,7 +639,7 @@ public void testGroupByExpressionMultiMultiAutoAutoWithFilter() new ExpressionVirtualColumn( "texpr", "concat(tags, othertags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -683,7 +683,7 @@ public void testGroupByExpressionAuto() new ExpressionVirtualColumn( "tt", "concat(tags, 'foo')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -725,12 +725,12 @@ public void testGroupByExpressionArrayExpressionFilter() .setDataSource("xx") .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("tt", "tt", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("tt", "tt", ColumnType.LONG)) .setVirtualColumns( new ExpressionVirtualColumn( "tt", "array_offset_of(tags, 't2')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) @@ -771,7 +771,7 @@ public void testGroupByExpressionArrayFnArg() new ExpressionVirtualColumn( "tt", "array_to_string(map(tags -> concat('foo', tags), tags), ', ')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -814,7 +814,7 @@ public void testGroupByExpressionAutoArrayFnArg() new ExpressionVirtualColumn( "tt", "array_to_string(concat('foo', tags), ', ')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -857,7 +857,7 @@ public void testGroupByExpressionFoldArrayToString() new ExpressionVirtualColumn( "tt", "fold((tag, acc) -> concat(acc, tag), tags, '')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -908,7 +908,7 @@ public void testGroupByExpressionFoldArrayToStringWithConcats() new ExpressionVirtualColumn( "tt", "fold((tag, acc) -> concat(concat(acc, case_searched(acc == '', '', ', '), concat('foo', tag)))), tags, '')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -952,7 +952,7 @@ public void testGroupByExpressionMultiConflicting() new ExpressionVirtualColumn( "texpr", "concat(map((x) -> concat(x, othertags), tags), tags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -987,7 +987,7 @@ public void testGroupByExpressionMultiConflictingAlso() new ExpressionVirtualColumn( "texpr", "array_concat(tags, (array_append(othertags, tags)))", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -1063,7 +1063,7 @@ public void testTopNExpression() new ExpressionVirtualColumn( "texpr", "map(x -> concat(x, 'foo'), tags)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -1120,7 +1120,7 @@ public void testTopNExpressionAutoTransform() new ExpressionVirtualColumn( "texpr", "concat(tags, 'foo')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregatorFactoryTest.java index 3aa9aa3ffeb8..0bc602c66421 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregatorFactoryTest.java @@ -43,8 +43,8 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -204,65 +204,65 @@ public void testResultArraySignature() RowSignature.builder() .addTimeColumn() // aggs - .add("count", ValueType.LONG) - .add("js", ValueType.FLOAT) - .add("longSum", ValueType.LONG) - .add("longMin", ValueType.LONG) - .add("longMax", ValueType.LONG) - .add("longFirst", ValueType.LONG) - .add("longLast", ValueType.LONG) - .add("longAny", ValueType.LONG) - .add("doubleSum", ValueType.DOUBLE) - .add("doubleMin", ValueType.DOUBLE) - .add("doubleMax", ValueType.DOUBLE) - .add("doubleFirst", ValueType.DOUBLE) - .add("doubleLast", ValueType.DOUBLE) - .add("doubleAny", ValueType.DOUBLE) + .add("count", ColumnType.LONG) + .add("js", ColumnType.FLOAT) + .add("longSum", ColumnType.LONG) + .add("longMin", ColumnType.LONG) + .add("longMax", ColumnType.LONG) + .add("longFirst", ColumnType.LONG) + .add("longLast", ColumnType.LONG) + .add("longAny", ColumnType.LONG) + .add("doubleSum", ColumnType.DOUBLE) + .add("doubleMin", ColumnType.DOUBLE) + .add("doubleMax", ColumnType.DOUBLE) + .add("doubleFirst", ColumnType.DOUBLE) + .add("doubleLast", ColumnType.DOUBLE) + .add("doubleAny", ColumnType.DOUBLE) .add("doubleMean", null) - .add("floatSum", ValueType.FLOAT) - .add("floatMin", ValueType.FLOAT) - .add("floatMax", ValueType.FLOAT) - .add("floatFirst", ValueType.FLOAT) - .add("floatLast", ValueType.FLOAT) - .add("floatAny", ValueType.FLOAT) + .add("floatSum", ColumnType.FLOAT) + .add("floatMin", ColumnType.FLOAT) + .add("floatMax", ColumnType.FLOAT) + .add("floatFirst", ColumnType.FLOAT) + .add("floatLast", ColumnType.FLOAT) + .add("floatAny", ColumnType.FLOAT) .add("stringFirst", null) .add("stringLast", null) - .add("stringAny", ValueType.STRING) + .add("stringAny", ColumnType.STRING) .add("cardinality", null) .add("hyperUnique", null) - .add("histogram", ValueType.COMPLEX) + .add("histogram", null) .add("filtered", null) .add("suppressed", null) // postaggs - .add("count-finalize", ValueType.LONG) - .add("js-finalize", ValueType.FLOAT) - .add("longSum-finalize", ValueType.LONG) - .add("longMin-finalize", ValueType.LONG) - .add("longMax-finalize", ValueType.LONG) - .add("longFirst-finalize", ValueType.LONG) - .add("longLast-finalize", ValueType.LONG) - .add("longAny-finalize", ValueType.LONG) - .add("doubleSum-finalize", ValueType.DOUBLE) - .add("doubleMin-finalize", ValueType.DOUBLE) - .add("doubleMax-finalize", ValueType.DOUBLE) - .add("doubleFirst-finalize", ValueType.DOUBLE) - .add("doubleLast-finalize", ValueType.DOUBLE) - .add("doubleAny-finalize", ValueType.DOUBLE) - .add("doubleMean-finalize", ValueType.DOUBLE) - .add("floatSum-finalize", ValueType.FLOAT) - .add("floatMin-finalize", ValueType.FLOAT) - .add("floatMax-finalize", ValueType.FLOAT) - .add("floatFirst-finalize", ValueType.FLOAT) - .add("floatLast-finalize", ValueType.FLOAT) - .add("floatAny-finalize", ValueType.FLOAT) - .add("stringFirst-finalize", ValueType.STRING) - .add("stringLast-finalize", ValueType.STRING) - .add("stringAny-finalize", ValueType.STRING) - .add("cardinality-finalize", ValueType.DOUBLE) - .add("hyperUnique-finalize", ValueType.DOUBLE) - .add("histogram-finalize", ValueType.COMPLEX) - .add("filtered-finalize", ValueType.DOUBLE) - .add("suppressed-finalize", ValueType.DOUBLE) + .add("count-finalize", ColumnType.LONG) + .add("js-finalize", ColumnType.FLOAT) + .add("longSum-finalize", ColumnType.LONG) + .add("longMin-finalize", ColumnType.LONG) + .add("longMax-finalize", ColumnType.LONG) + .add("longFirst-finalize", ColumnType.LONG) + .add("longLast-finalize", ColumnType.LONG) + .add("longAny-finalize", ColumnType.LONG) + .add("doubleSum-finalize", ColumnType.DOUBLE) + .add("doubleMin-finalize", ColumnType.DOUBLE) + .add("doubleMax-finalize", ColumnType.DOUBLE) + .add("doubleFirst-finalize", ColumnType.DOUBLE) + .add("doubleLast-finalize", ColumnType.DOUBLE) + .add("doubleAny-finalize", ColumnType.DOUBLE) + .add("doubleMean-finalize", ColumnType.DOUBLE) + .add("floatSum-finalize", ColumnType.FLOAT) + .add("floatMin-finalize", ColumnType.FLOAT) + .add("floatMax-finalize", ColumnType.FLOAT) + .add("floatFirst-finalize", ColumnType.FLOAT) + .add("floatLast-finalize", ColumnType.FLOAT) + .add("floatAny-finalize", ColumnType.FLOAT) + .add("stringFirst-finalize", ColumnType.STRING) + .add("stringLast-finalize", ColumnType.STRING) + .add("stringAny-finalize", ColumnType.STRING) + .add("cardinality-finalize", ColumnType.DOUBLE) + .add("hyperUnique-finalize", ColumnType.DOUBLE) + .add("histogram-finalize", HistogramAggregatorFactory.TYPE_VISUAL) + .add("filtered-finalize", ColumnType.DOUBLE) + .add("suppressed-finalize", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java index 3945c4feb20b..e0dc85a4450d 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -23,7 +23,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -73,7 +73,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("dblFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.DOUBLE).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.DOUBLE).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("dblFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java index 14702d1bf1f5..5b4041b84225 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java @@ -23,7 +23,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -74,7 +74,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("dblFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.DOUBLE).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.DOUBLE).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("dblFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java index f4e8a1485970..1c3b4c729417 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/ExpressionLambdaAggregatorFactoryTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; 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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -221,9 +221,9 @@ public void testStringType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.STRING, agg.getType()); - Assert.assertEquals(ValueType.STRING, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.STRING, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.STRING, agg.getType()); + Assert.assertEquals(ColumnType.STRING, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.STRING, agg.getFinalizedType()); } @Test @@ -244,9 +244,9 @@ public void testLongType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.LONG, agg.getType()); - Assert.assertEquals(ValueType.LONG, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.LONG, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.LONG, agg.getType()); + Assert.assertEquals(ColumnType.LONG, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.LONG, agg.getFinalizedType()); } @Test @@ -267,9 +267,9 @@ public void testDoubleType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.DOUBLE, agg.getType()); - Assert.assertEquals(ValueType.DOUBLE, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.DOUBLE, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.DOUBLE, agg.getType()); + Assert.assertEquals(ColumnType.DOUBLE, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.DOUBLE, agg.getFinalizedType()); } @Test @@ -290,9 +290,9 @@ public void testStringArrayType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.STRING, agg.getType()); - Assert.assertEquals(ValueType.STRING_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.STRING_ARRAY, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.STRING, agg.getType()); + Assert.assertEquals(ColumnType.STRING_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.STRING_ARRAY, agg.getFinalizedType()); } @Test @@ -313,9 +313,9 @@ public void testStringArrayTypeFinalized() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.STRING, agg.getType()); - Assert.assertEquals(ValueType.STRING_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.STRING, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.STRING, agg.getType()); + Assert.assertEquals(ColumnType.STRING_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.STRING, agg.getFinalizedType()); } @Test @@ -336,9 +336,9 @@ public void testLongArrayType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.LONG, agg.getType()); - Assert.assertEquals(ValueType.LONG_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.LONG_ARRAY, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.LONG, agg.getType()); + Assert.assertEquals(ColumnType.LONG_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.LONG_ARRAY, agg.getFinalizedType()); } @Test @@ -359,9 +359,9 @@ public void testLongArrayTypeFinalized() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.LONG, agg.getType()); - Assert.assertEquals(ValueType.LONG_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.STRING, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.LONG, agg.getType()); + Assert.assertEquals(ColumnType.LONG_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.STRING, agg.getFinalizedType()); } @Test @@ -382,9 +382,9 @@ public void testDoubleArrayType() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.DOUBLE, agg.getType()); - Assert.assertEquals(ValueType.DOUBLE_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.DOUBLE_ARRAY, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.DOUBLE, agg.getType()); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, agg.getFinalizedType()); } @Test @@ -405,9 +405,9 @@ public void testDoubleArrayTypeFinalized() TestExprMacroTable.INSTANCE ); - Assert.assertEquals(ValueType.DOUBLE, agg.getType()); - Assert.assertEquals(ValueType.DOUBLE_ARRAY, agg.getCombiningFactory().getType()); - Assert.assertEquals(ValueType.STRING, agg.getFinalizedType()); + Assert.assertEquals(ColumnType.DOUBLE, agg.getType()); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, agg.getCombiningFactory().getType()); + Assert.assertEquals(ColumnType.STRING, agg.getFinalizedType()); } @Test @@ -559,10 +559,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("string_expr", ValueType.STRING) - .add("double_expr", ValueType.DOUBLE) - .add("long_expr", ValueType.LONG) - .add("string_array_expr", ValueType.STRING_ARRAY) + .add("string_expr", ColumnType.STRING) + .add("double_expr", ColumnType.DOUBLE) + .add("long_expr", ColumnType.LONG) + .add("string_array_expr", ColumnType.STRING_ARRAY) // type does not equal finalized type. (combining factory type does equal finalized type, // but this signature doesn't use combining factory) .add("double_array_expr", null) @@ -570,24 +570,24 @@ public void testResultArraySignature() // but this signature doesn't use combining factory) .add("long_array_expr", null) // string because fold type equals finalized type, even though merge type is array - .add("string_array_expr_finalized", ValueType.STRING) + .add("string_array_expr_finalized", ColumnType.STRING) // type does not equal finalized type. (combining factory type does equal finalized type, // but this signature doesn't use combining factory) .add("double_array_expr_finalized", null) // long because fold type equals finalized type, even though merge type is array - .add("long_array_expr_finalized", ValueType.LONG) + .add("long_array_expr_finalized", ColumnType.LONG) // fold type is string - .add("string-array-expr-access", ValueType.STRING) + .add("string-array-expr-access", ColumnType.STRING) // finalized type is string - .add("string-array-expr-finalize", ValueType.STRING) + .add("string-array-expr-finalize", ColumnType.STRING) // double because fold type is double - .add("double-array-expr-access", ValueType.DOUBLE) + .add("double-array-expr-access", ColumnType.DOUBLE) // string because finalize type is string - .add("double-array-expr-finalize", ValueType.STRING) + .add("double-array-expr-finalize", ColumnType.STRING) // long because fold type is long - .add("long-array-expr-access", ValueType.LONG) + .add("long-array-expr-access", ColumnType.LONG) // finalized type is long - .add("long-array-expr-finalize", ValueType.LONG) + .add("long-array-expr-finalize", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java index 53eafd9f22a9..7c1d80959473 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java @@ -47,7 +47,7 @@ import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.SingleIndexedInt; import org.junit.Assert; @@ -202,12 +202,12 @@ public ColumnCapabilities getColumnCapabilities(String columnName) ColumnCapabilitiesImpl caps; if ("value".equals(columnName)) { caps = new ColumnCapabilitiesImpl(); - caps.setType(ValueType.FLOAT); + caps.setType(ColumnType.FLOAT); caps.setDictionaryEncoded(false); caps.setHasBitmapIndexes(false); } else { caps = new ColumnCapabilitiesImpl(); - caps.setType(ValueType.STRING); + caps.setType(ColumnType.STRING); caps.setDictionaryEncoded(true); caps.setHasBitmapIndexes(true); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/FloatMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/FloatMaxAggregationTest.java index dc1dc15c20a6..4b23e96b1b29 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/FloatMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/FloatMaxAggregationTest.java @@ -22,7 +22,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -59,7 +59,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("fltFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.FLOAT).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.FLOAT).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("fltFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/FloatMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/FloatMinAggregationTest.java index a451d1c4cc59..472892aa0e36 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/FloatMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/FloatMinAggregationTest.java @@ -22,7 +22,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -59,7 +59,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("fltFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.FLOAT).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.FLOAT).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("fltFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java index 838d3969cd68..d063658d76a2 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java @@ -23,7 +23,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -73,7 +73,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("lngFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.LONG).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.LONG).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("lngFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java index 00c9ab4f8c8f..f651ce074b93 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java @@ -23,7 +23,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.easymock.EasyMock; @@ -72,7 +72,7 @@ public void setup() vectorColumnSelectorFactory = EasyMock.createMock(VectorColumnSelectorFactory.class); EasyMock.expect(vectorColumnSelectorFactory.getColumnCapabilities("lngFld")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.LONG).setDictionaryEncoded(true)).anyTimes(); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.LONG).setDictionaryEncoded(true)).anyTimes(); EasyMock.expect(vectorColumnSelectorFactory.makeValueSelector("lngFld")).andReturn(vectorValueSelector).anyTimes(); EasyMock.replay(vectorColumnSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactoryTest.java index c923d40c8f9b..b36d10fac63a 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/any/DoubleAnyAggregatorFactoryTest.java @@ -23,7 +23,6 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -83,7 +82,7 @@ public void factorizeVectorShouldReturnDoubleVectorAggregator() public void factorizeVectorForNumericTypeShouldReturnDoubleVectorAggregator() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.DOUBLE).when(capabilities).getType(); + Mockito.doReturn(true).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(DoubleAnyVectorAggregator.class, aggregator.getClass()); @@ -93,7 +92,7 @@ public void factorizeVectorForNumericTypeShouldReturnDoubleVectorAggregator() public void factorizeVectorForStringTypeShouldReturnDoubleVectorAggregatorWithNilSelector() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.STRING).when(capabilities).getType(); + Mockito.doReturn(false).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(NullHandling.defaultDoubleValue(), aggregator.get(BUFFER, POSITION)); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactoryTest.java index 3ae93be2e7f5..9fa54cd8f592 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/any/FloatAnyAggregatorFactoryTest.java @@ -23,7 +23,6 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -83,7 +82,7 @@ public void factorizeVectorShouldReturnFloatVectorAggregator() public void factorizeVectorForNumericTypeShouldReturnFloatVectorAggregator() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.FLOAT).when(capabilities).getType(); + Mockito.doReturn(true).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(FloatAnyVectorAggregator.class, aggregator.getClass()); @@ -93,7 +92,7 @@ public void factorizeVectorForNumericTypeShouldReturnFloatVectorAggregator() public void factorizeVectorForStringTypeShouldReturnFloatVectorAggregatorWithNilSelector() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.STRING).when(capabilities).getType(); + Mockito.doReturn(false).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(NullHandling.defaultFloatValue(), aggregator.get(BUFFER, POSITION)); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactoryTest.java index d55b5ed039b5..9ae81f97bb11 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/any/LongAnyAggregatorFactoryTest.java @@ -23,7 +23,6 @@ import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -83,7 +82,7 @@ public void factorizeVectorShouldReturnLongVectorAggregator() public void factorizeVectorWithNumericColumnShouldReturnLongVectorAggregator() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.LONG).when(capabilities).getType(); + Mockito.doReturn(true).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(LongAnyVectorAggregator.class, aggregator.getClass()); @@ -93,7 +92,7 @@ public void factorizeVectorWithNumericColumnShouldReturnLongVectorAggregator() public void factorizeVectorForStringTypeShouldReturnLongVectorAggregatorWithNilSelector() { Mockito.doReturn(capabilities).when(selectorFactory).getColumnCapabilities(FIELD_NAME); - Mockito.doReturn(ValueType.STRING).when(capabilities).getType(); + Mockito.doReturn(false).when(capabilities).isNumeric(); VectorAggregator aggregator = target.factorizeVector(selectorFactory); Assert.assertNotNull(aggregator); Assert.assertEquals(NullHandling.defaultLongValue(), aggregator.get(BUFFER, POSITION)); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/any/StringAnyAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/any/StringAnyAggregationTest.java index 2d2d19e0d8f1..208cfeb052d1 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/any/StringAnyAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/any/StringAnyAggregationTest.java @@ -25,7 +25,7 @@ import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; @@ -57,7 +57,7 @@ public void setup() EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.STRING)); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.STRING)); EasyMock.expect(colSelectorFactory.getColumnCapabilities("billy")).andReturn(null); EasyMock.replay(colSelectorFactory); } @@ -83,7 +83,7 @@ public void testStringAnyAggregatorWithNullFirst() valueSelector = new TestObjectColumnSelector<>(stringsWithNullFirst); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.STRING)); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.STRING)); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.replay(colSelectorFactory); @@ -123,7 +123,7 @@ public void testStringAnyBufferAggregatorWithNullFirst() valueSelector = new TestObjectColumnSelector<>(stringsWithNullFirst); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.STRING)); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.STRING)); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.replay(colSelectorFactory); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstAggregationTest.java index 854cdbf9b1fd..8cebe4d88ef1 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstAggregationTest.java @@ -31,7 +31,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; @@ -74,7 +74,7 @@ public void setup() EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.STRING)); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.STRING)); EasyMock.expect(colSelectorFactory.getColumnCapabilities("billy")).andReturn(null); EasyMock.replay(colSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java index 8b7b085dec43..0dffb4d5daf6 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniqueFinalizingPostAggregatorTest.java @@ -30,8 +30,8 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -117,11 +117,11 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("approxCount", null) .add("approxCountRound", null) - .add("a", ValueType.DOUBLE) - .add("b", ValueType.LONG) + .add("a", ColumnType.DOUBLE) + .add("b", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastAggregationTest.java index dbd2522634e9..a08ff4c81390 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastAggregationTest.java @@ -31,7 +31,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Before; @@ -73,7 +73,7 @@ public void setup() EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(valueSelector); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("billy")).andReturn(objectSelector); EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")) - .andReturn(new ColumnCapabilitiesImpl().setType(ValueType.STRING)); + .andReturn(new ColumnCapabilitiesImpl().setType(ColumnType.STRING)); EasyMock.expect(colSelectorFactory.getColumnCapabilities("billy")).andReturn(null); EasyMock.replay(colSelectorFactory); } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregatorTest.java index d581a1103e1e..a93034427539 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/ArithmeticPostAggregatorTest.java @@ -32,8 +32,8 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -257,9 +257,9 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("sum", ValueType.LONG) - .add("count", ValueType.LONG) - .add("avg", ValueType.DOUBLE) + .add("sum", ColumnType.LONG) + .add("count", ColumnType.LONG) + .add("avg", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/ConstantPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/ConstantPostAggregatorTest.java index 12be9be56d62..4a3509cffdf5 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/ConstantPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/ConstantPostAggregatorTest.java @@ -25,8 +25,8 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -93,10 +93,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.LONG) - .add("b", ValueType.DOUBLE) - .add("c", ValueType.DOUBLE) + .add("count", ColumnType.LONG) + .add("a", ColumnType.LONG) + .add("b", ColumnType.DOUBLE) + .add("c", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java index 68f264ad5c27..48477c1fa45d 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleGreatestPostAggregatorTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -128,8 +128,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java index 3fc69ea4f4a2..96810fb0b945 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/DoubleLeastPostAggregatorTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -128,8 +128,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.DOUBLE) + .add("count", ColumnType.LONG) + .add("a", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregatorTest.java index 71f36fd33800..4748da7d7f1b 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/FieldAccessPostAggregatorTest.java @@ -27,8 +27,8 @@ import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -81,12 +81,12 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("double", ValueType.DOUBLE) - .add("float", ValueType.FLOAT) - .add("a", ValueType.LONG) - .add("b", ValueType.DOUBLE) - .add("c", ValueType.FLOAT) + .add("count", ColumnType.LONG) + .add("double", ColumnType.DOUBLE) + .add("float", ColumnType.FLOAT) + .add("a", ColumnType.LONG) + .add("b", ColumnType.DOUBLE) + .add("c", ColumnType.FLOAT) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java index f684d1497918..a60332208d58 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/FinalizingFieldAccessPostAggregatorTest.java @@ -40,8 +40,8 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; 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.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; @@ -85,7 +85,7 @@ public void testComputedWithFinalizing() AggregatorFactory aggFactory = EasyMock.createMock(AggregatorFactory.class); EasyMock.expect(aggFactory.getComparator()).andReturn(Comparators.naturalNullsFirst()).once(); EasyMock.expect(aggFactory.finalizeComputation("test")).andReturn(3L).once(); - EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ValueType.LONG).once(); + EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ColumnType.LONG).once(); EasyMock.replay(aggFactory); FinalizingFieldAccessPostAggregator postAgg = buildDecorated( @@ -109,7 +109,7 @@ public void testComputedInArithmeticPostAggregator() AggregatorFactory aggFactory = EasyMock.createMock(AggregatorFactory.class); EasyMock.expect(aggFactory.getComparator()).andReturn(Comparators.naturalNullsFirst()).once(); EasyMock.expect(aggFactory.finalizeComputation("test")).andReturn(3L).once(); - EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ValueType.LONG).once(); + EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ColumnType.LONG).once(); EasyMock.replay(aggFactory); FinalizingFieldAccessPostAggregator postAgg = buildDecorated( @@ -149,7 +149,7 @@ public void testComparatorsWithFinalizing() .andReturn(Ordering.natural().nullsLast()) .times(1); - EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ValueType.LONG).once(); + EasyMock.expect(aggFactory.getFinalizedType()).andReturn(ColumnType.LONG).once(); EasyMock.replay(aggFactory); FinalizingFieldAccessPostAggregator postAgg = buildDecorated( @@ -298,10 +298,10 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) + .add("count", ColumnType.LONG) .add("stringo", null) - .add("a", ValueType.COMPLEX) - .add("b", ValueType.STRING) + .add("a", StringFirstAggregatorFactory.TYPE) + .add("b", ColumnType.STRING) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregatorTest.java index fa38dec72c5a..e463e26f27ee 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/JavaScriptPostAggregatorTest.java @@ -26,8 +26,8 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -101,9 +101,9 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("total", ValueType.LONG) - .add("delta", ValueType.LONG) - .add("a", ValueType.DOUBLE) + .add("total", ColumnType.LONG) + .add("delta", ColumnType.LONG) + .add("a", ColumnType.DOUBLE) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java index 304978c069a7..5f93b09359a8 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/LongGreatestPostAggregatorTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -128,8 +128,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.LONG) + .add("count", ColumnType.LONG) + .add("a", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregatorTest.java index bc30588926dc..b97b52b5183a 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/post/LongLeastPostAggregatorTest.java @@ -28,8 +28,8 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -128,8 +128,8 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("count", ValueType.LONG) - .add("a", ValueType.LONG) + .add("count", ColumnType.LONG) + .add("a", ColumnType.LONG) .build(), new TimeseriesQueryQueryToolChest().resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/dimension/DefaultDimensionSpecTest.java b/processing/src/test/java/org/apache/druid/query/dimension/DefaultDimensionSpecTest.java index c0a251c7c713..a9178b539dfb 100644 --- a/processing/src/test/java/org/apache/druid/query/dimension/DefaultDimensionSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/dimension/DefaultDimensionSpecTest.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; @@ -48,7 +48,7 @@ public void testEqualsSerde() throws IOException public void testEqualsSerdeWithType() throws IOException { final String name = "foo"; - final DimensionSpec spec = new DefaultDimensionSpec(name, name, ValueType.FLOAT); + final DimensionSpec spec = new DefaultDimensionSpec(name, name, ColumnType.FLOAT); final String json = mapper.writeValueAsString(spec); final DimensionSpec other = mapper.readValue(json, DimensionSpec.class); Assert.assertEquals(spec.toString(), other.toString()); @@ -59,7 +59,7 @@ public void testEqualsSerdeWithType() throws IOException @Test public void testCacheKey() { - final DimensionSpec spec = new DefaultDimensionSpec("foo", "foo", ValueType.FLOAT); + final DimensionSpec spec = new DefaultDimensionSpec("foo", "foo", ColumnType.FLOAT); final byte[] expected = new byte[] {0, 7, 102, 111, 111, 7, 70, 76, 79, 65, 84}; Assert.assertArrayEquals(expected, spec.getCacheKey()); } diff --git a/processing/src/test/java/org/apache/druid/query/dimension/ExtractionDimensionSpecTest.java b/processing/src/test/java/org/apache/druid/query/dimension/ExtractionDimensionSpecTest.java index 453656268dc0..5027dc2eb383 100644 --- a/processing/src/test/java/org/apache/druid/query/dimension/ExtractionDimensionSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/dimension/ExtractionDimensionSpecTest.java @@ -24,7 +24,7 @@ import org.apache.druid.query.extraction.MatchingDimExtractionFn; import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.query.extraction.StrlenExtractionFn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; @@ -50,7 +50,7 @@ public void testSerde() throws Exception Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName()); Assert.assertEquals("myDim", extractionDimensionSpec.getDimension()); Assert.assertNotNull(extractionDimensionSpec.getExtractionFn()); - Assert.assertEquals(ValueType.STRING, extractionDimensionSpec.getOutputType()); + Assert.assertEquals(ColumnType.STRING, extractionDimensionSpec.getOutputType()); Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn); Assert.assertEquals( @@ -83,7 +83,7 @@ public void testSerdeWithType() throws Exception Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName()); Assert.assertEquals("myDim", extractionDimensionSpec.getDimension()); Assert.assertNotNull(extractionDimensionSpec.getExtractionFn()); - Assert.assertEquals(ValueType.LONG, extractionDimensionSpec.getOutputType()); + Assert.assertEquals(ColumnType.LONG, extractionDimensionSpec.getOutputType()); Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn); Assert.assertEquals( @@ -152,7 +152,7 @@ public void testCacheKey() final ExtractionDimensionSpec dimensionSpec = new ExtractionDimensionSpec( "foo", "len", - ValueType.LONG, + ColumnType.LONG, StrlenExtractionFn.instance() ); final byte[] expected = new byte[]{1, 7, 102, 111, 111, 9, 14, 7, 76, 79, 78, 71}; diff --git a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java index 25c27d449eef..0c933b67e913 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java @@ -31,8 +31,8 @@ import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; +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.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -217,7 +217,7 @@ public void testContainsNullWhenValuesSetIsTreeSet() final RowBasedColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), () -> new MapBasedRow(0, row), - RowSignature.builder().add("dim", ValueType.STRING).build(), + RowSignature.builder().add("dim", ColumnType.STRING).build(), true ); diff --git a/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java index 9f0e7c2faacc..c9ee77b0e971 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java @@ -22,7 +22,7 @@ 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.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorValueSelector; @@ -52,7 +52,7 @@ public void testFloat() { VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeFloatProcessor( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), vectorValueSelector ); @@ -80,7 +80,7 @@ public void testDouble() { VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeDoubleProcessor( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE), vectorValueSelector ); @@ -109,7 +109,7 @@ public void testLong() { VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeLongProcessor( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG), vectorValueSelector ); @@ -149,7 +149,7 @@ public void testSingleValueString() VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) @@ -187,7 +187,7 @@ public void testSingleValueStringZeroCardinalityAlwaysBooleanMatcher() VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) @@ -227,7 +227,7 @@ public void testSingleValueStringOneCardinalityBooleanMatcherIfNullAndNameLookup VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) @@ -270,7 +270,7 @@ public void testSingleValueStringOneCardinalityBooleanMatcherIfNullAndNameLookup VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) @@ -303,7 +303,7 @@ public void testMultiValueString() EasyMock.replay(selector, lookup); VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeMultiValueDimensionProcessor( - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(false) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 7cf7e65d2bb2..08365d50b4b9 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -77,7 +77,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -727,26 +727,26 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r new ExpressionVirtualColumn( "d0:v", "timestamp_extract(\"__time\",'YEAR','UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "d1:v", "timestamp_extract(\"__time\",'MONTH','UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( "d2:v", "timestamp_extract(\"__time\",'DAY','UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) .setDimensions( - new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG), - new DefaultDimensionSpec("d1:v", "d1", ValueType.LONG), - new DefaultDimensionSpec("d2:v", "d2", ValueType.LONG) + new DefaultDimensionSpec("d0:v", "d0", ColumnType.LONG), + new DefaultDimensionSpec("d1:v", "d1", ColumnType.LONG), + new DefaultDimensionSpec("d2:v", "d2", ColumnType.LONG) ).setAggregatorSpecs(new CountAggregatorFactory("a0")) .setLimitSpec( ls2 @@ -856,7 +856,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r new ExtractionDimensionSpec( ColumnHolder.TIME_COLUMN_NAME, "hour", - ValueType.LONG, + ColumnType.LONG, new TimeFormatExtractionFn( null, null, @@ -1000,8 +1000,8 @@ public Sequence run(QueryPlus queryPlus, ResponseContext r .setDataSource("blah") .setQuerySegmentSpec(intervalSpec) .setDimensions( - new DefaultDimensionSpec("dimA", "d0", ValueType.STRING), - new DefaultDimensionSpec("dimB", "d1", ValueType.STRING) + new DefaultDimensionSpec("dimA", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dimB", "d1", ColumnType.STRING) ).setAggregatorSpecs(new LongSumAggregatorFactory("a0", "metA")) .setLimitSpec(ls) .setContext(context) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index eaf85fc44ea8..36b0c5d080bd 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -64,6 +64,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; 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.virtual.ExpressionVirtualColumn; @@ -504,10 +505,10 @@ public void testResultLevelCacheKeyWithSubTotalsSpec() @Test public void testCacheStrategy() throws Exception { - doTestCacheStrategy(ValueType.STRING, "val1"); - doTestCacheStrategy(ValueType.FLOAT, 2.1f); - doTestCacheStrategy(ValueType.DOUBLE, 2.1d); - doTestCacheStrategy(ValueType.LONG, 2L); + doTestCacheStrategy(ColumnType.STRING, "val1"); + doTestCacheStrategy(ColumnType.FLOAT, 2.1f); + doTestCacheStrategy(ColumnType.DOUBLE, 2.1d); + doTestCacheStrategy(ColumnType.LONG, 2L); } @Test @@ -518,11 +519,11 @@ public void testMultiColumnCacheStrategy() throws Exception .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions(ImmutableList.of( - new DefaultDimensionSpec("test", "test", ValueType.STRING), - new DefaultDimensionSpec("v0", "v0", ValueType.STRING) + new DefaultDimensionSpec("test", "test", ColumnType.STRING), + new DefaultDimensionSpec("v0", "v0", ColumnType.STRING) )) .setVirtualColumns( - new ExpressionVirtualColumn("v0", "concat('foo', test)", ValueType.STRING, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("v0", "concat('foo', test)", ColumnType.STRING, TestExprMacroTable.INSTANCE) ) .setAggregatorSpecs( Arrays.asList( @@ -681,11 +682,11 @@ public void testResultArraySignatureAllGran() Assert.assertEquals( RowSignature.builder() - .add("dim", ValueType.STRING) - .add("rows", ValueType.LONG) - .add("index", ValueType.DOUBLE) + .add("dim", ColumnType.STRING) + .add("rows", ColumnType.LONG) + .add("index", ColumnType.DOUBLE) .add("uniques", null) - .add("const", ValueType.LONG) + .add("const", ColumnType.LONG) .build(), new GroupByQueryQueryToolChest(null, null).resultArraySignature(query) ); @@ -706,11 +707,11 @@ public void testResultArraySignatureDayGran() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("dim", ValueType.STRING) - .add("rows", ValueType.LONG) - .add("index", ValueType.DOUBLE) + .add("dim", ColumnType.STRING) + .add("rows", ColumnType.LONG) + .add("index", ColumnType.DOUBLE) .add("uniques", null) - .add("const", ValueType.LONG) + .add("const", ColumnType.LONG) .build(), new GroupByQueryQueryToolChest(null, null).resultArraySignature(query) ); @@ -862,7 +863,7 @@ private SerializablePair getIntermediateComplexValue(final ValueType valueType, } } - private void doTestCacheStrategy(final ValueType valueType, final Object dimValue) throws IOException + private void doTestCacheStrategy(final ColumnType valueType, final Object dimValue) throws IOException { final GroupByQuery query1 = GroupByQuery .builder() @@ -874,7 +875,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu .setAggregatorSpecs( Arrays.asList( QueryRunnerTestHelper.ROWS_COUNT, - getComplexAggregatorFactoryForValueType(valueType) + getComplexAggregatorFactoryForValueType(valueType.getType()) ) ) .setPostAggregatorSpecs( @@ -889,7 +890,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu ); // test timestamps that result in integer size millis - final ResultRow result1 = ResultRow.of(123L, dimValue, 1, getIntermediateComplexValue(valueType, dimValue)); + final ResultRow result1 = ResultRow.of(123L, dimValue, 1, getIntermediateComplexValue(valueType.getType(), dimValue)); Object preparedValue = strategy.prepareForSegmentLevelCache().apply(result1); @@ -908,9 +909,9 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu // Please see the comments on aggregator serde and type handling in CacheStrategy.fetchAggregatorsFromCache() final ResultRow typeAdjustedResult2; - if (valueType == ValueType.FLOAT) { + if (valueType.is(ValueType.FLOAT)) { typeAdjustedResult2 = ResultRow.of(123L, dimValue, 1, 2.1d, 10); - } else if (valueType == ValueType.LONG) { + } else if (valueType.is(ValueType.LONG)) { typeAdjustedResult2 = ResultRow.of(123L, dimValue, 1, 2, 10); } else { typeAdjustedResult2 = result2; diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index 2d73c713d086..2401adc8a11e 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -128,7 +128,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -1025,7 +1025,7 @@ public void testGroupByWithStringVirtualColumn() new ExpressionVirtualColumn( "vc", "quality + 'x'", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -1088,7 +1088,7 @@ public void testGroupByWithStringVirtualColumnVectorizable() new ExpressionVirtualColumn( "vc", "cast(quality, 'STRING')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -3278,7 +3278,7 @@ public void testMergeResultsAcrossMultipleDaysWithLimitAndOrderByUsingMathExpres new ExpressionVirtualColumn( "expr", "index * 2 + indexMin / 10", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -3488,7 +3488,7 @@ public void testGroupByOrderLimit() ExpressionVirtualColumn expressionVirtualColumn = new ExpressionVirtualColumn( "expr", "index / 2 + indexMin", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); List aggregatorSpecs2 = Arrays.asList( @@ -4897,7 +4897,7 @@ public void testDifferentGroupingSubquery() subquery = makeQueryBuilder(subquery) .setVirtualColumns( - new ExpressionVirtualColumn("expr", "-index + 100", ValueType.FLOAT, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("expr", "-index + 100", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ) .setAggregatorSpecs( QueryRunnerTestHelper.ROWS_COUNT, @@ -5965,7 +5965,7 @@ public void testSubqueryWithOuterVirtualColumns() final GroupByQuery query = makeQueryBuilder() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setVirtualColumns(new ExpressionVirtualColumn("expr", "1", ValueType.FLOAT, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn("expr", "1", ColumnType.FLOAT, TestExprMacroTable.INSTANCE)) .setDimensions(new ArrayList<>()).setAggregatorSpecs(new LongSumAggregatorFactory("count", "expr")) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) .build(); @@ -6401,7 +6401,7 @@ public void testGroupByWithSubtotalsSpecOfDimensionsPrefixes() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ColumnType.STRING, TestExprMacroTable.INSTANCE)) .setDimensions(Lists.newArrayList( new DefaultDimensionSpec("market", "market2"), new DefaultDimensionSpec("alias", "alias2") @@ -6514,7 +6514,7 @@ public void testGroupByWithSubtotalsSpecGeneral() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ColumnType.STRING, TestExprMacroTable.INSTANCE)) .setDimensions(Lists.newArrayList( new DefaultDimensionSpec("quality", "quality2"), new DefaultDimensionSpec("market", "market2"), @@ -6870,7 +6870,7 @@ public void testGroupByWithSubtotalsSpecWithRenamedDimensionAndFilter() .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE)) + .setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ColumnType.STRING, TestExprMacroTable.INSTANCE)) .setDimensions(Lists.newArrayList( new DefaultDimensionSpec("quality", "quality"), new DefaultDimensionSpec("market", "market"), @@ -6963,7 +6963,7 @@ public void testGroupByWithSubtotalsSpecWithLongDimensionColumn() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions(Lists.newArrayList( - new DefaultDimensionSpec("qualityLong", "ql", ValueType.LONG), + new DefaultDimensionSpec("qualityLong", "ql", ColumnType.LONG), new DefaultDimensionSpec("market", "market2") )) .setAggregatorSpecs( @@ -8862,7 +8862,7 @@ public void testGroupByCardinalityAggOnMultiStringExpression() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setVirtualColumns( - new ExpressionVirtualColumn("v0", "concat(quality,market)", ValueType.STRING, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("v0", "concat(quality,market)", ColumnType.STRING, TestExprMacroTable.INSTANCE) ) .setAggregatorSpecs( QueryRunnerTestHelper.ROWS_COUNT, @@ -8938,7 +8938,7 @@ public void testGroupByLongColumn() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG)) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .addOrderByColumn(new OrderByColumnSpec( @@ -9018,7 +9018,7 @@ public void testGroupByLongColumnDescending() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG)) .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .addOrderByColumn(new OrderByColumnSpec( @@ -9117,7 +9117,7 @@ public void testGroupByLongTimeColumn() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("__time", "time_alias", ColumnType.LONG)) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .setGranularity(QueryRunnerTestHelper.DAY_GRAN) @@ -9204,7 +9204,7 @@ public void testGroupByFloatColumn() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT)) + .setDimensions(new DefaultDimensionSpec("index", "index_alias", ColumnType.FLOAT)) .setDimFilter(new SelectorDimFilter("quality", "entertainment", null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .addOrderByColumn(new OrderByColumnSpec( @@ -9255,7 +9255,7 @@ public void testGroupByFloatColumnDescending() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT)) + .setDimensions(new DefaultDimensionSpec("qualityFloat", "qf_alias", ColumnType.FLOAT)) .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .addOrderByColumn(new OrderByColumnSpec( @@ -9305,7 +9305,7 @@ public void testGroupByDoubleColumnDescending() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setDimensions(new DefaultDimensionSpec("qualityDouble", "alias", ValueType.DOUBLE)) + .setDimensions(new DefaultDimensionSpec("qualityDouble", "alias", ColumnType.DOUBLE)) .setDimFilter(new InDimFilter("quality", Arrays.asList("entertainment", "technology"), null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .addOrderByColumn(new OrderByColumnSpec( @@ -9409,9 +9409,9 @@ public void testGroupByWithHavingSpecOnLongAndFloat() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("market", "alias"), - new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), - new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG), - new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT) + new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG), + new DefaultDimensionSpec("__time", "time_alias", ColumnType.LONG), + new DefaultDimensionSpec("index", "index_alias", ColumnType.FLOAT) ).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setHavingSpec( new DimFilterHavingSpec( @@ -9528,10 +9528,10 @@ public void testGroupByNumericStringsAsNumeric() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG), - new DefaultDimensionSpec("ql_alias", "ql_alias_long", ValueType.LONG), - new DefaultDimensionSpec("qf_alias", "qf_alias_float", ValueType.FLOAT), - new DefaultDimensionSpec("ql_alias", "ql_alias_float", ValueType.FLOAT) + new DefaultDimensionSpec("time_alias", "time_alias2", ColumnType.LONG), + new DefaultDimensionSpec("ql_alias", "ql_alias_long", ColumnType.LONG), + new DefaultDimensionSpec("qf_alias", "qf_alias_float", ColumnType.FLOAT), + new DefaultDimensionSpec("ql_alias", "ql_alias_float", ColumnType.FLOAT) ).setAggregatorSpecs(new CountAggregatorFactory("count")) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) .build(); @@ -9574,12 +9574,12 @@ public void testGroupByNumericStringsAsNumericWithDecoration() // rows with `technology` have `170000` in the qualityNumericString field RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( - new DefaultDimensionSpec("qualityNumericString", "ql", ValueType.LONG), + new DefaultDimensionSpec("qualityNumericString", "ql", ColumnType.LONG), "170000" ); ListFilteredDimensionSpec listFilteredSpec = new ListFilteredDimensionSpec( - new DefaultDimensionSpec("qualityNumericString", "qf", ValueType.FLOAT), + new DefaultDimensionSpec("qualityNumericString", "qf", ColumnType.FLOAT), Sets.newHashSet("170000"), true ); @@ -9629,12 +9629,12 @@ public void testGroupByDecorationOnNumerics() } RegexFilteredDimensionSpec regexSpec = new RegexFilteredDimensionSpec( - new DefaultDimensionSpec("qualityLong", "ql", ValueType.LONG), + new DefaultDimensionSpec("qualityLong", "ql", ColumnType.LONG), "1700" ); ListFilteredDimensionSpec listFilteredSpec = new ListFilteredDimensionSpec( - new DefaultDimensionSpec("qualityFloat", "qf", ValueType.FLOAT), + new DefaultDimensionSpec("qualityFloat", "qf", ColumnType.FLOAT), Sets.newHashSet("17000.0"), true ); @@ -9701,8 +9701,8 @@ public void testGroupByNestedWithInnerQueryNumerics() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("quality", "alias"), - new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), - new DefaultDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT) + new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG), + new DefaultDimensionSpec("qualityFloat", "qf_alias", ColumnType.FLOAT) ) .setDimFilter( new InDimFilter( @@ -9718,8 +9718,8 @@ public void testGroupByNestedWithInnerQueryNumerics() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("ql_alias", "quallong", ValueType.LONG), - new DefaultDimensionSpec("qf_alias", "qualfloat", ValueType.FLOAT) + new DefaultDimensionSpec("ql_alias", "quallong", ColumnType.LONG), + new DefaultDimensionSpec("qf_alias", "qualfloat", ColumnType.FLOAT) ) .setDimFilter( new AndDimFilter( @@ -9781,9 +9781,9 @@ public void testGroupByNestedWithInnerQueryOutputNullNumerics() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("quality", "alias"), - new ExtractionDimensionSpec("qualityLong", "ql_alias", ValueType.LONG, extractionFn), - new ExtractionDimensionSpec("qualityFloat", "qf_alias", ValueType.FLOAT, extractionFn), - new ExtractionDimensionSpec("qualityDouble", "qd_alias", ValueType.DOUBLE, extractionFn) + new ExtractionDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG, extractionFn), + new ExtractionDimensionSpec("qualityFloat", "qf_alias", ColumnType.FLOAT, extractionFn), + new ExtractionDimensionSpec("qualityDouble", "qd_alias", ColumnType.DOUBLE, extractionFn) ) .setDimFilter( new InDimFilter( @@ -9799,9 +9799,9 @@ public void testGroupByNestedWithInnerQueryOutputNullNumerics() .setDataSource(subquery) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("ql_alias", "quallong", ValueType.LONG), - new DefaultDimensionSpec("qf_alias", "qualfloat", ValueType.FLOAT), - new DefaultDimensionSpec("qd_alias", "qualdouble", ValueType.DOUBLE) + new DefaultDimensionSpec("ql_alias", "quallong", ColumnType.LONG), + new DefaultDimensionSpec("qf_alias", "qualfloat", ColumnType.FLOAT), + new DefaultDimensionSpec("qd_alias", "qualdouble", ColumnType.DOUBLE) ) .setAggregatorSpecs( new LongSumAggregatorFactory("ql_alias_sum", "ql_alias"), @@ -9863,8 +9863,8 @@ public void testGroupByNestedWithInnerQueryNumericsWithLongTime() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("market", "alias"), - new DefaultDimensionSpec("__time", "time_alias", ValueType.LONG), - new DefaultDimensionSpec("index", "index_alias", ValueType.FLOAT) + new DefaultDimensionSpec("__time", "time_alias", ColumnType.LONG), + new DefaultDimensionSpec("index", "index_alias", ColumnType.FLOAT) ).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) .build(); @@ -9874,7 +9874,7 @@ public void testGroupByNestedWithInnerQueryNumericsWithLongTime() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("alias", "market"), - new DefaultDimensionSpec("time_alias", "time_alias2", ValueType.LONG) + new DefaultDimensionSpec("time_alias", "time_alias2", ColumnType.LONG) ) .setAggregatorSpecs( new LongMaxAggregatorFactory("time_alias_max", "time_alias"), @@ -9957,7 +9957,7 @@ public void testGroupByStringOutputAsLong() .setDimensions(new ExtractionDimensionSpec( QueryRunnerTestHelper.QUALITY_DIMENSION, "alias", - ValueType.LONG, + ColumnType.LONG, strlenFn )) .setDimFilter(new SelectorDimFilter(QueryRunnerTestHelper.QUALITY_DIMENSION, "entertainment", null)) @@ -10080,7 +10080,7 @@ public void testGroupByNestedOuterExtractionFnOnFloatInner() .setDimensions(new DefaultDimensionSpec("quality", "alias"), new ExtractionDimensionSpec( "qualityFloat", "qf_inner", - ValueType.FLOAT, + ColumnType.FLOAT, jsExtractionFn )) .setDimFilter(new SelectorDimFilter("quality", "technology", null)) @@ -10094,7 +10094,7 @@ public void testGroupByNestedOuterExtractionFnOnFloatInner() .setDimensions(new DefaultDimensionSpec("alias", "alias"), new ExtractionDimensionSpec( "qf_inner", "qf_outer", - ValueType.FLOAT, + ColumnType.FLOAT, jsExtractionFn )).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -10127,7 +10127,7 @@ public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() new ExtractionDimensionSpec( ColumnHolder.TIME_COLUMN_NAME, "time_day", - ValueType.LONG, + ColumnType.LONG, new TimeFormatExtractionFn(null, null, null, Granularities.DAY, true) ) ) @@ -10142,7 +10142,7 @@ public void testGroupByNestedDoubleTimeExtractionFnWithLongOutputTypes() .setDimensions(new DefaultDimensionSpec("alias", "alias"), new ExtractionDimensionSpec( "time_day", "time_week", - ValueType.LONG, + ColumnType.LONG, new TimeFormatExtractionFn(null, null, null, Granularities.WEEK, true) )).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -10265,7 +10265,7 @@ public void testGroupByLimitPushDownWithLongDimensionNotInLimitSpec() GroupByQuery query = makeQueryBuilder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setGranularity(QueryRunnerTestHelper.ALL_GRAN).setDimensions( - new ExtractionDimensionSpec("quality", "qualityLen", ValueType.LONG, StrlenExtractionFn.instance()) + new ExtractionDimensionSpec("quality", "qualityLen", ColumnType.LONG, StrlenExtractionFn.instance()) ) .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) .setLimitSpec( @@ -10869,7 +10869,7 @@ public void testVirtualColumnFilterOnInnerQuery() new ExpressionVirtualColumn( "v", "case_searched(idx > 1000, 1, 0)", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ) @@ -10948,7 +10948,7 @@ public void testTypeConversionWithMergingChainedExecutionRunner() .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( new DefaultDimensionSpec("quality", "alias"), - new ExtractionDimensionSpec("quality", "qualityLen", ValueType.LONG, StrlenExtractionFn.instance()) + new ExtractionDimensionSpec("quality", "qualityLen", ColumnType.LONG, StrlenExtractionFn.instance()) ) .setDimFilter(new SelectorDimFilter("quality", "technology", null)) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) @@ -11014,7 +11014,7 @@ public void testGroupByOnNullableLong() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("longNumericNull", "nullable", ValueType.LONG) + new DefaultDimensionSpec("longNumericNull", "nullable", ColumnType.LONG) ) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -11055,7 +11055,7 @@ public void testGroupByOnNullableDouble() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("doubleNumericNull", "nullable", ValueType.DOUBLE) + new DefaultDimensionSpec("doubleNumericNull", "nullable", ColumnType.DOUBLE) ) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -11096,7 +11096,7 @@ public void testGroupByOnNullableDoubleNoLimitPushdown() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("doubleNumericNull", "nullable", ValueType.DOUBLE) + new DefaultDimensionSpec("doubleNumericNull", "nullable", ColumnType.DOUBLE) ) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -11138,7 +11138,7 @@ public void testGroupByOnNullableFloat() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) .setDimensions( - new DefaultDimensionSpec("floatNumericNull", "nullable", ValueType.FLOAT) + new DefaultDimensionSpec("floatNumericNull", "nullable", ColumnType.FLOAT) ) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -11182,7 +11182,7 @@ public void testGroupByOnVirtualColumn() new ExpressionVirtualColumn( "v", "qualityDouble * qualityLong", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ), new ExpressionVirtualColumn( @@ -11193,7 +11193,7 @@ public void testGroupByOnVirtualColumn() ) ) .setDimensions( - new DefaultDimensionSpec("v", "v", ValueType.LONG) + new DefaultDimensionSpec("v", "v", ColumnType.LONG) ) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("twosum", null, "1 + two", TestExprMacroTable.INSTANCE)) .setGranularity(QueryRunnerTestHelper.ALL_GRAN) @@ -11452,7 +11452,7 @@ public void testGroupByWithExpressionAggregatorWithArrays() // array types don't work with group by v1 if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Unable to handle type[STRING_ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); + expectedException.expectMessage("Unable to handle type[ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); } GroupByQuery query = makeQueryBuilder() @@ -11739,7 +11739,7 @@ public void testGroupByExpressionAggregatorArrayMultiValue() // array types don't work with group by v1 if (config.getDefaultStrategy().equals(GroupByStrategySelector.STRATEGY_V1)) { expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Unable to handle type[STRING_ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); + expectedException.expectMessage("Unable to handle type[ARRAY] for AggregatorFactory[class org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory]"); } GroupByQuery query = makeQueryBuilder() diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java index cec90ed2df28..186a43c27af8 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryTest.java @@ -41,7 +41,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.junit.Assert; import org.junit.Test; @@ -90,7 +90,7 @@ public void testGetRequiredColumns() .builder() .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) - .setVirtualColumns(new ExpressionVirtualColumn("v", "\"other\"", ValueType.STRING, ExprMacroTable.nil())) + .setVirtualColumns(new ExpressionVirtualColumn("v", "\"other\"", ColumnType.STRING, ExprMacroTable.nil())) .setDimensions(new DefaultDimensionSpec("quality", "alias"), DefaultDimensionSpec.of("v")) .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .setGranularity(QueryRunnerTestHelper.DAY_GRAN) @@ -117,9 +117,9 @@ public void testRowOrderingMixTypes() .setDataSource("dummy") .setGranularity(Granularities.ALL) .setInterval("2000/2001") - .addDimension(new DefaultDimensionSpec("foo", "foo", ValueType.LONG)) - .addDimension(new DefaultDimensionSpec("bar", "bar", ValueType.FLOAT)) - .addDimension(new DefaultDimensionSpec("baz", "baz", ValueType.STRING)) + .addDimension(new DefaultDimensionSpec("foo", "foo", ColumnType.LONG)) + .addDimension(new DefaultDimensionSpec("bar", "bar", ColumnType.FLOAT)) + .addDimension(new DefaultDimensionSpec("baz", "baz", ColumnType.STRING)) .build(); final Ordering rowOrdering = query.getRowOrdering(false); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 03ef00673723..a8dc0617c779 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -48,7 +48,7 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.joda.time.DateTime; import org.junit.AfterClass; @@ -121,7 +121,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) new ExpressionVirtualColumn( "v0", StringUtils.format("timestamp_floor(__time, '%s')", granularity.getPeriod()), - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ) ); @@ -140,7 +140,7 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) .setDimensions( timeDimension == null ? ImmutableList.of() - : ImmutableList.of(new DefaultDimensionSpec("v0", timeDimension, ValueType.LONG)) + : ImmutableList.of(new DefaultDimensionSpec("v0", timeDimension, ColumnType.LONG)) ) .setAggregatorSpecs(tsQuery.getAggregatorSpecs()) .setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs()) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java index 75a12a90cb6c..d5d26d0c50bf 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java @@ -22,6 +22,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.easymock.EasyMock; import org.junit.Assert; @@ -42,7 +43,7 @@ public void setUp() @Test public void testCanPushDownLimitForSegmentStringSelector() { - ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasBitmapIndexes(true) .setHasMultipleValues(false) .setDictionaryEncoded(true) @@ -57,7 +58,7 @@ public void testCanPushDownLimitForSegmentStringSelector() @Test public void testCanPushDownLimitForIncrementalStringSelector() { - ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasBitmapIndexes(false) .setHasMultipleValues(false) .setDictionaryEncoded(false) @@ -72,7 +73,7 @@ public void testCanPushDownLimitForIncrementalStringSelector() @Test public void testCanPushDownLimitForExpressionStringSelector() { - ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasBitmapIndexes(false) .setHasMultipleValues(false) .setDictionaryEncoded(false) @@ -87,7 +88,7 @@ public void testCanPushDownLimitForExpressionStringSelector() @Test public void testCanPushDownLimitForJoinStringSelector() { - ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilities capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasBitmapIndexes(false) .setHasMultipleValues(false) .setDictionaryEncoded(true) @@ -102,7 +103,7 @@ public void testCanPushDownLimitForJoinStringSelector() @Test public void testCanPushDownLimitForNumericSelector() { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.LONG) + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.LONG) .setHasBitmapIndexes(false) .setHasMultipleValues(false) .setDictionaryEncoded(false) @@ -111,9 +112,9 @@ public void testCanPushDownLimitForNumericSelector() EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).anyTimes(); EasyMock.replay(factory); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); - capabilities.setType(ValueType.DOUBLE); + capabilities.setType(ColumnType.DOUBLE); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); - capabilities.setType(ValueType.FLOAT); + capabilities.setType(ColumnType.FLOAT); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); EasyMock.verify(factory); } @@ -121,7 +122,7 @@ public void testCanPushDownLimitForNumericSelector() @Test public void testCanPushDownLimitForComplexSelector() { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX) + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(new ColumnType(ValueType.COMPLEX, "foo", null)) .setHasBitmapIndexes(false) .setHasMultipleValues(false) .setDictionaryEncoded(false) diff --git a/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java index f0d82d67185c..93bc3e641223 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/having/DimFilterHavingSpecTest.java @@ -26,7 +26,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Ignore; @@ -73,7 +73,7 @@ public void testRowSignature() .setDataSource("dummy") .setInterval("1000/3000") .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("foo", "foo", ValueType.LONG)) + .setDimensions(new DefaultDimensionSpec("foo", "foo", ColumnType.LONG)) .build() ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java b/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java index 51daf2661cbb..3029b2f2b693 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/orderby/DefaultLimitSpecTest.java @@ -34,7 +34,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -190,7 +190,7 @@ public void testWithAllGranularity() GroupByQuery.builder() .setDataSource("dummy") .setInterval("1000/3000") - .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setDimensions(new DefaultDimensionSpec("k1", "k1", ColumnType.DOUBLE)) .setGranularity(Granularities.ALL) .overrideContext(ImmutableMap.of(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true)) .build() @@ -214,7 +214,7 @@ public void testWithSortByDimsFirst() GroupByQuery.builder() .setDataSource("dummy") .setInterval("1000/3000") - .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setDimensions(new DefaultDimensionSpec("k1", "k1", ColumnType.DOUBLE)) .setGranularity(Granularities.NONE) .overrideContext(ImmutableMap.of(GroupByQuery.CTX_KEY_SORT_BY_DIMS_FIRST, true)) .build() @@ -238,7 +238,7 @@ public void testSortDimensionDescending() GroupByQuery.builder() .setDataSource("dummy") .setInterval("1000/3000") - .setDimensions(new DefaultDimensionSpec("k1", "k1", ValueType.DOUBLE)) + .setDimensions(new DefaultDimensionSpec("k1", "k1", ColumnType.DOUBLE)) .setGranularity(Granularities.ALL) .build() ); diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 11e00f76a505..8dd259148f34 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -47,6 +47,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -105,9 +106,9 @@ private void testIncrementalWorksHelper(EnumSet an if ("null_column".equals(dimension)) { Assert.assertNull(columnAnalysis); } else { - final boolean isString = schema.getValueType().name().equals(ValueType.STRING.name()); - Assert.assertEquals(dimension, schema.getValueType().name(), columnAnalysis.getType()); + final boolean isString = schema.getColumnType().is(ValueType.STRING); + Assert.assertEquals(dimension, schema.getColumnType().toString(), columnAnalysis.getType()); Assert.assertEquals(dimension, 0, columnAnalysis.getSize()); if (isString) { @@ -415,15 +416,9 @@ public List requiredFields() } @Override - public ValueType getType() + public ColumnType getType() { - return ValueType.COMPLEX; - } - - @Override - public String getComplexTypeName() - { - return TYPE; + return new ColumnType(ValueType.COMPLEX, TYPE, null); } @Override @@ -439,7 +434,7 @@ public byte[] getCacheKey() } @Override - public ValueType getFinalizedType() + public ColumnType getFinalizedType() { return getType(); } diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index 7d226fb5a9f5..3e6acc1b62fe 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -461,7 +461,7 @@ public void testSegmentMetadataQueryWithComplexColumnMerge() ), "quality_uniques", new ColumnAnalysis( - "hyperUnique", + "COMPLEX", false, true, 0, diff --git a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java index 93e75c3f190c..aab7d0e0baa2 100644 --- a/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java +++ b/processing/src/test/java/org/apache/druid/query/planning/DataSourceAnalysisTest.java @@ -36,8 +36,8 @@ import org.apache.druid.query.filter.TrueDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +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.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; import org.joda.time.Interval; @@ -56,7 +56,7 @@ public class DataSourceAnalysisTest private static final LookupDataSource LOOKUP_LOOKYLOO = new LookupDataSource("lookyloo"); private static final InlineDataSource INLINE = InlineDataSource.fromIterable( ImmutableList.of(new Object[0]), - RowSignature.builder().add("column", ValueType.STRING).build() + RowSignature.builder().add("column", ColumnType.STRING).build() ); @Test diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java index e1bbc71d29bc..002d80957e87 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryResultOrderingTest.java @@ -40,8 +40,8 @@ import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.column.ColumnHolder; +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.timeline.SegmentId; import org.joda.time.DateTime; import org.junit.Assert; @@ -95,7 +95,7 @@ public Function columnFunction(String columnName) private static final RowSignature ROW_SIGNATURE = RowSignature.builder() .addTimeColumn() - .add(ID_COLUMN, ValueType.LONG) + .add(ID_COLUMN, ColumnType.LONG) .build(); private static final List> SEGMENTS = ImmutableList.of( diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index a08d23b216f3..3e148c7f5e63 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -82,7 +82,7 @@ public class ScanQueryRunnerTest extends InitializedNullHandlingTest { private static final VirtualColumn EXPR_COLUMN = - new ExpressionVirtualColumn("expr", "index * 2", ValueType.LONG, TestExprMacroTable.INSTANCE); + new ExpressionVirtualColumn("expr", "index * 2", ColumnType.LONG, TestExprMacroTable.INSTANCE); // Read the first set of 12 lines from the sample data, which covers the day 2011-01-12T00:00:00.000Z public static final String[] V_0112 = readLinesFromSample(0, 13).toArray(new String[0]); diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index 16a3590d9974..6656d8caa399 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -629,7 +629,7 @@ public void testSearchOnLongColumn() new DefaultDimensionSpec( ColumnHolder.TIME_COLUMN_NAME, ColumnHolder.TIME_COLUMN_NAME, - ValueType.LONG + ColumnType.LONG ) ) .dataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -676,7 +676,7 @@ public void testSearchOnFloatColumn() new DefaultDimensionSpec( QueryRunnerTestHelper.INDEX_METRIC, QueryRunnerTestHelper.INDEX_METRIC, - ValueType.DOUBLE + ColumnType.DOUBLE ) ) .dataSource(QueryRunnerTestHelper.DATA_SOURCE) diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index 483d907b7dc4..2a877fb0d03e 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -43,8 +43,8 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -380,10 +380,10 @@ public void testResultArraySignatureWithoutTimestampResultField() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("rows", ValueType.LONG) - .add("index", ValueType.DOUBLE) + .add("rows", ColumnType.LONG) + .add("index", ColumnType.DOUBLE) .add("uniques", null) - .add("const", ValueType.LONG) + .add("const", ColumnType.LONG) .build(), TOOL_CHEST.resultArraySignature(query) ); @@ -406,11 +406,11 @@ public void testResultArraySignatureWithTimestampResultField() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add(TIMESTAMP_RESULT_FIELD_NAME, ValueType.LONG) - .add("rows", ValueType.LONG) - .add("index", ValueType.DOUBLE) + .add(TIMESTAMP_RESULT_FIELD_NAME, ColumnType.LONG) + .add("rows", ColumnType.LONG) + .add("index", ColumnType.DOUBLE) .add("uniques", null) - .add("const", ValueType.LONG) + .add("const", ColumnType.LONG) .build(), TOOL_CHEST.resultArraySignature(query) ); diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 63e5aa7ca27d..2a8408c2901c 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -70,8 +70,8 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; 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.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -695,7 +695,7 @@ public void testTimeseriesWithVirtualColumn() new ExpressionVirtualColumn( "expr", "index", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ) ) @@ -3122,7 +3122,7 @@ public void testTimeseriesWithExpressionAggregatorTooBig() cannotVectorize(); if (!vectorize) { // size bytes when it overshoots varies slightly between algorithms - expectedException.expectMessage("Unable to serialize [STRING_ARRAY]"); + expectedException.expectMessage("Unable to serialize [ARRAY]"); } TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) @@ -3160,7 +3160,7 @@ public void testTimeseriesCardinalityAggOnMultiStringExpression() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD) .virtualColumns( - new ExpressionVirtualColumn("v0", "concat(quality,market)", ValueType.STRING, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("v0", "concat(quality,market)", ColumnType.STRING, TestExprMacroTable.INSTANCE) ) .aggregators( QueryRunnerTestHelper.ROWS_COUNT, diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java index 54bebf8df406..e8f10da8bf3d 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.junit.Assert; import org.junit.Test; @@ -84,7 +84,7 @@ public void testGetRequiredColumns() new ExpressionVirtualColumn( "index", "\"fieldFromVirtualColumn\"", - ValueType.LONG, + ColumnType.LONG, ExprMacroTable.nil() ) ) diff --git a/processing/src/test/java/org/apache/druid/query/topn/DimValHolderTest.java b/processing/src/test/java/org/apache/druid/query/topn/DimValHolderTest.java index 5087890f80ae..e71d353e933e 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/DimValHolderTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/DimValHolderTest.java @@ -19,7 +19,7 @@ package org.apache.druid.query.topn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; @@ -30,40 +30,40 @@ public void testDimTypeConversion() { DimValHolder.Builder builder = new DimValHolder.Builder(); - builder.withDimValue("1", ValueType.STRING); + builder.withDimValue("1", ColumnType.STRING); Assert.assertEquals("1", builder.build().getDimValue()); - builder.withDimValue("1", ValueType.LONG); + builder.withDimValue("1", ColumnType.LONG); Assert.assertEquals(1L, builder.build().getDimValue()); - builder.withDimValue("1", ValueType.FLOAT); + builder.withDimValue("1", ColumnType.FLOAT); Assert.assertEquals(1f, builder.build().getDimValue()); - builder.withDimValue("1", ValueType.DOUBLE); + builder.withDimValue("1", ColumnType.DOUBLE); Assert.assertEquals(1d, builder.build().getDimValue()); - builder.withDimValue(1L, ValueType.STRING); + builder.withDimValue(1L, ColumnType.STRING); Assert.assertEquals("1", builder.build().getDimValue()); - builder.withDimValue(1L, ValueType.LONG); + builder.withDimValue(1L, ColumnType.LONG); Assert.assertEquals(1L, builder.build().getDimValue()); - builder.withDimValue(1L, ValueType.FLOAT); + builder.withDimValue(1L, ColumnType.FLOAT); Assert.assertEquals(1f, builder.build().getDimValue()); - builder.withDimValue(1L, ValueType.DOUBLE); + builder.withDimValue(1L, ColumnType.DOUBLE); Assert.assertEquals(1d, builder.build().getDimValue()); - builder.withDimValue(1f, ValueType.STRING); + builder.withDimValue(1f, ColumnType.STRING); Assert.assertEquals("1.0", builder.build().getDimValue()); - builder.withDimValue(1f, ValueType.LONG); + builder.withDimValue(1f, ColumnType.LONG); Assert.assertEquals(1L, builder.build().getDimValue()); - builder.withDimValue(1f, ValueType.FLOAT); + builder.withDimValue(1f, ColumnType.FLOAT); Assert.assertEquals(1f, builder.build().getDimValue()); - builder.withDimValue(1f, ValueType.DOUBLE); + builder.withDimValue(1f, ColumnType.DOUBLE); Assert.assertEquals(1d, builder.build().getDimValue()); - builder.withDimValue(1d, ValueType.STRING); + builder.withDimValue(1d, ColumnType.STRING); Assert.assertEquals("1.0", builder.build().getDimValue()); - builder.withDimValue(1d, ValueType.LONG); + builder.withDimValue(1d, ColumnType.LONG); Assert.assertEquals(1L, builder.build().getDimValue()); - builder.withDimValue(1d, ValueType.FLOAT); + builder.withDimValue(1d, ColumnType.FLOAT); Assert.assertEquals(1f, builder.build().getDimValue()); - builder.withDimValue(1d, ValueType.DOUBLE); + builder.withDimValue(1d, ColumnType.DOUBLE); Assert.assertEquals(1d, builder.build().getDimValue()); } } diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java index 18e7f361712d..790e426216cd 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -61,6 +61,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.VirtualColumns; +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.testing.InitializedNullHandlingTest; @@ -90,19 +91,19 @@ public static void setUpClass() @Test public void testCacheStrategy() throws Exception { - doTestCacheStrategy(ValueType.STRING, "val1"); - doTestCacheStrategy(ValueType.FLOAT, 2.1f); - doTestCacheStrategy(ValueType.DOUBLE, 2.1d); - doTestCacheStrategy(ValueType.LONG, 2L); + doTestCacheStrategy(ColumnType.STRING, "val1"); + doTestCacheStrategy(ColumnType.FLOAT, 2.1f); + doTestCacheStrategy(ColumnType.DOUBLE, 2.1d); + doTestCacheStrategy(ColumnType.LONG, 2L); } @Test public void testCacheStrategyOrderByPostAggs() throws Exception { - doTestCacheStrategyOrderByPost(ValueType.STRING, "val1"); - doTestCacheStrategyOrderByPost(ValueType.FLOAT, 2.1f); - doTestCacheStrategyOrderByPost(ValueType.DOUBLE, 2.1d); - doTestCacheStrategyOrderByPost(ValueType.LONG, 2L); + doTestCacheStrategyOrderByPost(ColumnType.STRING, "val1"); + doTestCacheStrategyOrderByPost(ColumnType.FLOAT, 2.1f); + doTestCacheStrategyOrderByPost(ColumnType.DOUBLE, 2.1d); + doTestCacheStrategyOrderByPost(ColumnType.LONG, 2L); } @Test @@ -315,11 +316,11 @@ public void testResultArraySignature() Assert.assertEquals( RowSignature.builder() .addTimeColumn() - .add("dim", ValueType.STRING) - .add("rows", ValueType.LONG) - .add("index", ValueType.DOUBLE) + .add("dim", ColumnType.STRING) + .add("rows", ColumnType.LONG) + .add("index", ColumnType.DOUBLE) .add("uniques", null) - .add("const", ValueType.LONG) + .add("const", ColumnType.LONG) .build(), new TopNQueryQueryToolChest(null, null).resultArraySignature(query) ); @@ -420,7 +421,7 @@ private HyperLogLogCollector getIntermediateHllCollector(final ValueType valueTy return collector; } - private void doTestCacheStrategy(final ValueType valueType, final Object dimValue) throws IOException + private void doTestCacheStrategy(final ColumnType valueType, final Object dimValue) throws IOException { CacheStrategy, Object, TopNQuery> strategy = new TopNQueryQueryToolChest(null, null).getCacheStrategy( @@ -435,7 +436,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu Granularities.ALL, ImmutableList.of( new CountAggregatorFactory("metric1"), - getComplexAggregatorFactoryForValueType(valueType) + getComplexAggregatorFactoryForValueType(valueType.getType()) ), ImmutableList.of(new ConstantPostAggregator("post", 10)), null @@ -450,7 +451,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu ImmutableMap.of( "test", dimValue, "metric1", 2, - "complexMetric", getIntermediateComplexValue(valueType, dimValue) + "complexMetric", getIntermediateComplexValue(valueType.getType(), dimValue) ) ) ) @@ -487,7 +488,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu // Please see the comments on aggregator serde and type handling in CacheStrategy.fetchAggregatorsFromCache() final Result typeAdjustedResult2; - if (valueType == ValueType.FLOAT) { + if (valueType.is(ValueType.FLOAT)) { typeAdjustedResult2 = new Result<>( DateTimes.utc(123L), new TopNResultValue( @@ -501,7 +502,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu ) ) ); - } else if (valueType == ValueType.LONG) { + } else if (valueType.is(ValueType.LONG)) { typeAdjustedResult2 = new Result<>( DateTimes.utc(123L), new TopNResultValue( @@ -533,7 +534,7 @@ private void doTestCacheStrategy(final ValueType valueType, final Object dimValu Assert.assertEquals(typeAdjustedResult2, fromResultCacheResult); } - private void doTestCacheStrategyOrderByPost(final ValueType valueType, final Object dimValue) throws IOException + private void doTestCacheStrategyOrderByPost(final ColumnType valueType, final Object dimValue) throws IOException { CacheStrategy, Object, TopNQuery> strategy = new TopNQueryQueryToolChest(null, null).getCacheStrategy( @@ -570,7 +571,7 @@ private void doTestCacheStrategyOrderByPost(final ValueType valueType, final Obj ) ); - HyperLogLogCollector collector = getIntermediateHllCollector(valueType, dimValue); + HyperLogLogCollector collector = getIntermediateHllCollector(valueType.getType(), dimValue); final Result result1 = new Result<>( // test timestamps that result in integer size millis diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index c24b104f4530..23f724ae96a3 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -92,7 +92,7 @@ import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; @@ -2246,7 +2246,7 @@ public void testTopNDimExtractionTimeToOneLong() new ExtractionDimensionSpec( ColumnHolder.TIME_COLUMN_NAME, "t", - ValueType.LONG, + ColumnType.LONG, new JavaScriptExtractionFn( "function(f) { return \"42\"; }", false, @@ -4256,7 +4256,7 @@ public void testFullOnTopNFloatColumn() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec(QueryRunnerTestHelper.INDEX_METRIC, "index_alias", ValueType.FLOAT)) + .dimension(new DefaultDimensionSpec(QueryRunnerTestHelper.INDEX_METRIC, "index_alias", ColumnType.FLOAT)) .metric(QueryRunnerTestHelper.INDEX_METRIC) .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -4475,7 +4475,7 @@ public void testFullOnTopNLongColumn() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -4547,7 +4547,7 @@ public void testFullOnTopNLongVirtualColumn() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("ql_expr", "ql_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("ql_expr", "ql_alias", ColumnType.LONG)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -4563,7 +4563,7 @@ public void testFullOnTopNLongVirtualColumn() ) ) .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) - .virtualColumns(new ExpressionVirtualColumn("ql_expr", "qualityLong", ValueType.LONG, ExprMacroTable.nil())) + .virtualColumns(new ExpressionVirtualColumn("ql_expr", "qualityLong", ColumnType.LONG, ExprMacroTable.nil())) .build(); List> expectedResults = Collections.singletonList( @@ -4624,7 +4624,7 @@ public void testTopNStringVirtualColumn() new ExpressionVirtualColumn( "vc", "market + ' ' + market", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -4822,7 +4822,7 @@ public void testFullOnTopNNumericStringColumnAsLong() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ColumnType.LONG)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -4894,7 +4894,7 @@ public void testFullOnTopNNumericStringColumnAsFloat() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.FLOAT)) + .dimension(new DefaultDimensionSpec("qualityNumericString", "qns_alias", ColumnType.FLOAT)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -4966,7 +4966,7 @@ public void testFullOnTopNLongTimeColumn() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, "time_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec(ColumnHolder.TIME_COLUMN_NAME, "time_alias", ColumnType.LONG)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5038,7 +5038,7 @@ public void testSortOnDoubleAsLong() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("index", "index_alias", ColumnType.LONG)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5074,7 +5074,7 @@ public void testSortOnTimeAsLong() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("__time", "__time_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("__time", "__time_alias", ColumnType.LONG)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5110,7 +5110,7 @@ public void testSortOnStringAsDouble() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("market", "alias", ValueType.DOUBLE)) + .dimension(new DefaultDimensionSpec("market", "alias", ColumnType.DOUBLE)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5134,7 +5134,7 @@ public void testSortOnDoubleAsDouble() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.DOUBLE)) + .dimension(new DefaultDimensionSpec("index", "index_alias", ColumnType.DOUBLE)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5300,7 +5300,7 @@ public void testFullOnTopNStringOutputAsLong() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.QUALITY_DIMENSION, "alias", ValueType.LONG, strlenFn)) + .dimension(new ExtractionDimensionSpec(QueryRunnerTestHelper.QUALITY_DIMENSION, "alias", ColumnType.LONG, strlenFn)) .metric("maxIndex") .threshold(4) .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) @@ -5370,7 +5370,7 @@ public void testFullOnTopNStringOutputAsLong() public void testFullOnTopNNumericStringColumnWithDecoration() { ListFilteredDimensionSpec filteredSpec = new ListFilteredDimensionSpec( - new DefaultDimensionSpec("qualityNumericString", "qns_alias", ValueType.LONG), + new DefaultDimensionSpec("qualityNumericString", "qns_alias", ColumnType.LONG), Sets.newHashSet("120000", "140000", "160000"), true ); @@ -5439,7 +5439,7 @@ public void testFullOnTopNNumericStringColumnWithDecoration() public void testFullOnTopNDecorationOnNumeric() { ListFilteredDimensionSpec filteredSpec = new ListFilteredDimensionSpec( - new DefaultDimensionSpec("qualityLong", "ql_alias", ValueType.LONG), + new DefaultDimensionSpec("qualityLong", "ql_alias", ColumnType.LONG), Sets.newHashSet("1200", "1400", "1600"), true ); @@ -5672,7 +5672,7 @@ public void test_topN_orderByLongNumericColumnWithNulls_returnsDescendingResults TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("index", "index_alias", ColumnType.LONG)) .metric(new NumericTopNMetricSpec("longNumericNull")) .threshold(10000) .aggregators(new LongSumAggregatorFactory("longNumericNull", "longNumericNull")) @@ -5740,7 +5740,7 @@ public void test_topN_orderByFloatNumericColumnWithNulls_returnsDescendingResult TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("index", "index_alias", ColumnType.LONG)) .metric(new NumericTopNMetricSpec("floatNumericNull")) .threshold(10000) .aggregators(new LongSumAggregatorFactory("floatNumericNull", "floatNumericNull")) @@ -5808,7 +5808,7 @@ public void test_topN_orderByDoubleNumericColumnWithNulls_returnsDescendingResul TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("index", "index_alias", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("index", "index_alias", ColumnType.LONG)) .metric(new NumericTopNMetricSpec("doubleNumericNull")) .threshold(10000) .aggregators(new LongSumAggregatorFactory("doubleNumericNull", "doubleNumericNull")) @@ -5877,7 +5877,7 @@ public void testAggregateOnLongNumericNull() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("longNumericNull", "dim", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("longNumericNull", "dim", ColumnType.LONG)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(10000) .aggregators(new CountAggregatorFactory("count")) @@ -5909,7 +5909,7 @@ public void testAggregateOnDoubleNumericNull() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("doubleNumericNull", "dim", ValueType.DOUBLE)) + .dimension(new DefaultDimensionSpec("doubleNumericNull", "dim", ColumnType.DOUBLE)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(10000) .aggregators(new CountAggregatorFactory("count")) @@ -5941,7 +5941,7 @@ public void testAggregateOnFloatNumericNull() TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .granularity(QueryRunnerTestHelper.ALL_GRAN) - .dimension(new DefaultDimensionSpec("floatNumericNull", "dim", ValueType.FLOAT)) + .dimension(new DefaultDimensionSpec("floatNumericNull", "dim", ColumnType.FLOAT)) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(10000) .aggregators(new CountAggregatorFactory("count")) diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 5aede90d29fa..6d7f1a03be9a 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -39,7 +39,7 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.junit.Assert; import org.junit.Rule; @@ -255,7 +255,7 @@ public void testGetRequiredColumns() final TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.DATA_SOURCE) .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD) - .virtualColumns(new ExpressionVirtualColumn("v", "\"other\"", ValueType.STRING, ExprMacroTable.nil())) + .virtualColumns(new ExpressionVirtualColumn("v", "\"other\"", ColumnType.STRING, ExprMacroTable.nil())) .dimension(DefaultDimensionSpec.of("v")) .aggregators(QueryRunnerTestHelper.ROWS_COUNT, new LongSumAggregatorFactory("idx", "index")) .granularity(QueryRunnerTestHelper.DAY_GRAN) diff --git a/processing/src/test/java/org/apache/druid/segment/DimensionHandlerUtilsTest.java b/processing/src/test/java/org/apache/druid/segment/DimensionHandlerUtilsTest.java index ee600feae662..a953fc3065c6 100644 --- a/processing/src/test/java/org/apache/druid/segment/DimensionHandlerUtilsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/DimensionHandlerUtilsTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.junit.Assert; import org.junit.Test; @@ -36,7 +37,7 @@ public void testRegisterDimensionHandlerProvider() DimensionHandler dimensionHandler = DimensionHandlerUtils.getHandlerFromCapabilities( "dim", - new ColumnCapabilitiesImpl().setType(ValueType.COMPLEX).setComplexTypeName("testType"), + new ColumnCapabilitiesImpl().setType(new ColumnType(ValueType.COMPLEX, "testType", null)), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index f52827efac33..0f4ecddde99e 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -222,7 +222,7 @@ public RowBasedSegment buildRowBasedSegmentWithTypeSignature() final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); for (final String columnName : index.getColumnNames()) { final ColumnCapabilities capabilities = index.getColumnHolder(columnName).getCapabilities(); - rowSignatureBuilder.add(columnName, capabilities.getType()); + rowSignatureBuilder.add(columnName, capabilities.toColumnType()); } return new RowBasedSegment<>( diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java index cdfc1b149781..163f2390c6f6 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -158,25 +159,25 @@ public static void teardown() public void testNumericColumns() { // incremental index - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities(ColumnHolder.TIME_COLUMN_NAME), ValueType.LONG); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d3"), ValueType.DOUBLE); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d4"), ValueType.FLOAT); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d5"), ValueType.LONG); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m1"), ValueType.DOUBLE); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m2"), ValueType.FLOAT); - assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m3"), ValueType.LONG); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities(ColumnHolder.TIME_COLUMN_NAME), ColumnType.LONG); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d3"), ColumnType.DOUBLE); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d4"), ColumnType.FLOAT); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("d5"), ColumnType.LONG); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m1"), ColumnType.DOUBLE); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m2"), ColumnType.FLOAT); + assertNonStringColumnCapabilities(INC_INDEX.getCapabilities("m3"), ColumnType.LONG); // segment index assertNonStringColumnCapabilities( MMAP_INDEX.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getCapabilities(), - ValueType.LONG + ColumnType.LONG ); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d3").getCapabilities(), ValueType.DOUBLE); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d4").getCapabilities(), ValueType.FLOAT); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d5").getCapabilities(), ValueType.LONG); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m1").getCapabilities(), ValueType.DOUBLE); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m2").getCapabilities(), ValueType.FLOAT); - assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m3").getCapabilities(), ValueType.LONG); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d3").getCapabilities(), ColumnType.DOUBLE); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d4").getCapabilities(), ColumnType.FLOAT); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("d5").getCapabilities(), ColumnType.LONG); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m1").getCapabilities(), ColumnType.DOUBLE); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m2").getCapabilities(), ColumnType.FLOAT); + assertNonStringColumnCapabilities(MMAP_INDEX.getColumnHolder("m3").getCapabilities(), ColumnType.LONG); } @Test @@ -186,43 +187,43 @@ public void testNumericColumnsWithNulls() // time does not have nulls assertNonStringColumnCapabilities( INC_INDEX_WITH_NULLS.getCapabilities(ColumnHolder.TIME_COLUMN_NAME), - ValueType.LONG + ColumnType.LONG ); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d3"), ValueType.DOUBLE); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d4"), ValueType.FLOAT); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d5"), ValueType.LONG); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m1"), ValueType.DOUBLE); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m2"), ValueType.FLOAT); - assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m3"), ValueType.LONG); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d3"), ColumnType.DOUBLE); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d4"), ColumnType.FLOAT); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("d5"), ColumnType.LONG); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m1"), ColumnType.DOUBLE); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m2"), ColumnType.FLOAT); + assertNonStringColumnCapabilitiesWithNulls(INC_INDEX_WITH_NULLS.getCapabilities("m3"), ColumnType.LONG); // segment index assertNonStringColumnCapabilities( MMAP_INDEX_WITH_NULLS.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME).getCapabilities(), - ValueType.LONG + ColumnType.LONG ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("d3").getCapabilities(), - ValueType.DOUBLE + ColumnType.DOUBLE ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("d4").getCapabilities(), - ValueType.FLOAT + ColumnType.FLOAT ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("d5").getCapabilities(), - ValueType.LONG + ColumnType.LONG ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("m1").getCapabilities(), - ValueType.DOUBLE + ColumnType.DOUBLE ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("m2").getCapabilities(), - ValueType.FLOAT + ColumnType.FLOAT ); assertNonStringColumnCapabilitiesWithNulls( MMAP_INDEX_WITH_NULLS.getColumnHolder("m3").getCapabilities(), - ValueType.LONG + ColumnType.LONG ); } @@ -355,7 +356,7 @@ public void testComplexColumn() private void assertComplexColumnCapabilites(ColumnCapabilities caps) { - Assert.assertEquals(ValueType.COMPLEX, caps.getType()); + Assert.assertEquals(HyperUniquesAggregatorFactory.TYPE, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); @@ -365,9 +366,9 @@ private void assertComplexColumnCapabilites(ColumnCapabilities caps) Assert.assertTrue(caps.hasNulls().isTrue()); } - private void assertNonStringColumnCapabilities(ColumnCapabilities caps, ValueType valueType) + private void assertNonStringColumnCapabilities(ColumnCapabilities caps, ColumnType valueType) { - Assert.assertEquals(valueType, caps.getType()); + Assert.assertEquals(valueType, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); @@ -377,9 +378,9 @@ private void assertNonStringColumnCapabilities(ColumnCapabilities caps, ValueTyp Assert.assertFalse(caps.hasNulls().isTrue()); } - private void assertNonStringColumnCapabilitiesWithNulls(ColumnCapabilities caps, ValueType valueType) + private void assertNonStringColumnCapabilitiesWithNulls(ColumnCapabilities caps, ColumnType valueType) { - Assert.assertEquals(valueType, caps.getType()); + Assert.assertEquals(valueType, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java index 26967d5b1c5d..6550f550ad85 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java @@ -21,6 +21,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +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.testing.InitializedNullHandlingTest; @@ -37,17 +38,18 @@ public class RowBasedColumnSelectorFactoryTest extends InitializedNullHandlingTe private static final String DOUBLE_ARRAY_COLUMN_NAME = "double_array"; private static final String LONG_ARRAY_COLUMN_NAME = "long_array"; private static final String STRING_ARRAY_COLUMN_NAME = "string_array"; + private static final ColumnType SOME_COMPLEX = new ColumnType(ValueType.COMPLEX, "foo", null); private static final RowSignature ROW_SIGNATURE = RowSignature.builder() - .add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG) - .add(STRING_COLUMN_NAME, ValueType.STRING) - .add(LONG_COLUMN_NAME, ValueType.LONG) - .add(FLOAT_COLUMN_NAME, ValueType.FLOAT) - .add(DOUBLE_COLUMN_NAME, ValueType.DOUBLE) - .add(COMPLEX_COLUMN_NAME, ValueType.COMPLEX) - .add(DOUBLE_ARRAY_COLUMN_NAME, ValueType.DOUBLE_ARRAY) - .add(LONG_ARRAY_COLUMN_NAME, ValueType.LONG_ARRAY) - .add(STRING_ARRAY_COLUMN_NAME, ValueType.STRING_ARRAY) + .add(ColumnHolder.TIME_COLUMN_NAME, ColumnType.LONG) + .add(STRING_COLUMN_NAME, ColumnType.STRING) + .add(LONG_COLUMN_NAME, ColumnType.LONG) + .add(FLOAT_COLUMN_NAME, ColumnType.FLOAT) + .add(DOUBLE_COLUMN_NAME, ColumnType.DOUBLE) + .add(COMPLEX_COLUMN_NAME, SOME_COMPLEX) + .add(DOUBLE_ARRAY_COLUMN_NAME, ColumnType.DOUBLE_ARRAY) + .add(LONG_ARRAY_COLUMN_NAME, ColumnType.LONG_ARRAY) + .add(STRING_ARRAY_COLUMN_NAME, ColumnType.STRING_ARRAY) .build(); @Test @@ -126,7 +128,7 @@ public void testCapabilitiesComplex() { ColumnCapabilities caps = RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, COMPLEX_COLUMN_NAME); - Assert.assertEquals(ValueType.COMPLEX, caps.getType()); + Assert.assertEquals(SOME_COMPLEX, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); @@ -140,7 +142,7 @@ public void testCapabilitiesDoubleArray() { ColumnCapabilities caps = RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, DOUBLE_ARRAY_COLUMN_NAME); - Assert.assertEquals(ValueType.DOUBLE_ARRAY, caps.getType()); + Assert.assertEquals(ColumnType.DOUBLE_ARRAY, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); @@ -154,7 +156,7 @@ public void testCapabilitiesLongArray() { ColumnCapabilities caps = RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, LONG_ARRAY_COLUMN_NAME); - Assert.assertEquals(ValueType.LONG_ARRAY, caps.getType()); + Assert.assertEquals(ColumnType.LONG_ARRAY, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); @@ -168,7 +170,7 @@ public void testCapabilitiesStringArray() { ColumnCapabilities caps = RowBasedColumnSelectorFactory.getColumnCapabilities(ROW_SIGNATURE, STRING_ARRAY_COLUMN_NAME); - Assert.assertEquals(ValueType.STRING_ARRAY, caps.getType()); + Assert.assertEquals(ColumnType.STRING_ARRAY, caps.toColumnType()); Assert.assertFalse(caps.hasBitmapIndexes()); Assert.assertFalse(caps.isDictionaryEncoded().isTrue()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index 0bd58aa4058b..21d35bcd6c66 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -33,6 +33,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -59,11 +60,11 @@ public class RowBasedStorageAdapterTest private static final RowSignature ROW_SIGNATURE = RowSignature.builder() - .add(ValueType.FLOAT.name(), ValueType.FLOAT) - .add(ValueType.DOUBLE.name(), ValueType.DOUBLE) - .add(ValueType.LONG.name(), ValueType.LONG) - .add(ValueType.STRING.name(), ValueType.STRING) - .add(ValueType.COMPLEX.name(), ValueType.COMPLEX) + .add(ValueType.FLOAT.name(), ColumnType.FLOAT) + .add(ValueType.DOUBLE.name(), ColumnType.DOUBLE) + .add(ValueType.LONG.name(), ColumnType.LONG) + .add(ValueType.STRING.name(), ColumnType.STRING) + .add(ValueType.COMPLEX.name(), ColumnType.UNKNOWN_COMPLEX) .add(UNKNOWN_TYPE_NAME, null) .build(); @@ -197,7 +198,7 @@ public Function columnFunction(String columnName) if (valueType == null || valueType == ValueType.COMPLEX) { return i -> null; } else { - return i -> DimensionHandlerUtils.convertObjectToType(i, valueType); + return i -> DimensionHandlerUtils.convertObjectToType(i, ROW_SIGNATURE.getColumnType(columnName).orElse(null)); } } } @@ -377,7 +378,7 @@ public void test_getColumnCapabilities_complex() final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.COMPLEX.name()); // Note: unlike numeric types, COMPLEX-typed columns report that they are incomplete. - Assert.assertEquals(ValueType.COMPLEX, capabilities.getType()); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, capabilities.toColumnType()); Assert.assertTrue(capabilities.hasMultipleValues().isUnknown()); } @@ -499,7 +500,7 @@ public void test_makeCursors_filterOnVirtualColumn() new ExpressionVirtualColumn( "vc", "\"LONG\" + 1", - ValueType.LONG, + ColumnType.LONG, ExprMacroTable.nil() ) ) diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index 020a4ecb89d0..ad97940c0be0 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -46,7 +46,7 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -139,7 +139,7 @@ public class TestIndex private static final Logger log = new Logger(TestIndex.class); private static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create( Collections.singletonList( - new ExpressionVirtualColumn("expr", "index + 10", ValueType.FLOAT, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("expr", "index + 10", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ) ); public static final AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ diff --git a/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java b/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java index 30af23fc80f8..cb2cb95d5df4 100644 --- a/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java +++ b/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java @@ -36,7 +36,7 @@ public void testSerde() throws Exception .setHasBitmapIndexes(true) .setHasMultipleValues(true) .setHasSpatialIndexes(true) - .setType(ValueType.COMPLEX) + .setType(ColumnType.UNKNOWN_COMPLEX) .setHasNulls(true) .setFilterable(true)); @@ -44,7 +44,7 @@ public void testSerde() throws Exception ColumnCapabilities cc = mapper.readValue(json, ColumnCapabilitiesImpl.class); - Assert.assertEquals(ValueType.COMPLEX, cc.getType()); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, cc.toColumnType()); Assert.assertTrue(cc.isDictionaryEncoded().isTrue()); Assert.assertTrue(cc.hasSpatialIndexes()); Assert.assertTrue(cc.hasMultipleValues().isTrue()); @@ -71,7 +71,7 @@ public void testDeserialization() throws Exception ColumnCapabilities cc = mapper.readValue(json, ColumnCapabilitiesImpl.class); - Assert.assertEquals(ValueType.COMPLEX, cc.getType()); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, cc.toColumnType()); Assert.assertTrue(cc.isDictionaryEncoded().isTrue()); Assert.assertTrue(cc.hasSpatialIndexes()); Assert.assertTrue(cc.hasMultipleValues().isTrue()); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 67d1c24fbe62..7e21bcf61983 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -47,6 +47,7 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Parser; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.aggregation.Aggregator; @@ -74,8 +75,8 @@ import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; +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.BitmapSerdeFactory; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.data.IndexedInts; @@ -117,14 +118,14 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest static final VirtualColumns VIRTUAL_COLUMNS = VirtualColumns.create( ImmutableList.of( - new ExpressionVirtualColumn("expr", "1.0 + 0.1", ValueType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("exprDouble", "1.0 + 1.1", ValueType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("exprLong", "1 + 2", ValueType.LONG, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim0", "dim0", ValueType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vdim1", "dim1", ValueType.STRING, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vd0", "d0", ValueType.DOUBLE, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vf0", "f0", ValueType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("vl0", "l0", ValueType.LONG, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("expr", "1.0 + 0.1", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("exprDouble", "1.0 + 1.1", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("exprLong", "1 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vdim0", "dim0", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vdim1", "dim1", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE) ) ); @@ -150,13 +151,13 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest // missing 'dim3' because makeDefaultSchemaRow does not expect to set it... static final RowSignature DEFAULT_ROW_SIGNATURE = RowSignature.builder() - .add("dim0", ValueType.STRING) - .add("dim1", ValueType.STRING) - .add("dim2", ValueType.STRING) - .add("timeDim", ValueType.STRING) - .add("d0", ValueType.DOUBLE) - .add("f0", ValueType.FLOAT) - .add("l0", ValueType.LONG) + .add("dim0", ColumnType.STRING) + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .add("timeDim", ColumnType.STRING) + .add("d0", ColumnType.DOUBLE) + .add("f0", ColumnType.FLOAT) + .add("l0", ColumnType.LONG) .build(); static final List DEFAULT_ROWS = ImmutableList.of( @@ -687,10 +688,10 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur final Expr parsedIdentifier = Parser.parse(selectColumn, TestExprMacroTable.INSTANCE); try (final VectorCursor cursor = makeVectorCursor(makeFilter(filter))) { - final ExprType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); + final ExpressionType outputType = parsedIdentifier.getOutputType(cursor.getColumnSelectorFactory()); final List values = new ArrayList<>(); - if (ExprType.STRING.equals(outputType)) { + if (outputType.is(ExprType.STRING)) { final VectorObjectSelector objectSelector = cursor.getColumnSelectorFactory().makeObjectSelector( virtualColumn ); @@ -705,7 +706,7 @@ private List selectColumnValuesMatchingFilterUsingVectorVirtualColumnCur final VectorValueSelector valueSelector = cursor.getColumnSelectorFactory().makeValueSelector(virtualColumn); while (!cursor.isDone()) { final boolean[] nulls = valueSelector.getNullVector(); - if (ExprType.DOUBLE.equals(outputType)) { + if (outputType.is(ExprType.DOUBLE)) { final double[] doubles = valueSelector.getDoubleVector(); for (int i = 0; i < cursor.getCurrentVectorSize(); i++) { if (nulls != null && nulls[i]) { @@ -743,7 +744,7 @@ private List selectColumnValuesMatchingFilterUsingRowBasedColumnSelector // Generate rowSignature final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); for (String columnName : Iterables.concat(adapter.getAvailableDimensions(), adapter.getAvailableMetrics())) { - rowSignatureBuilder.add(columnName, adapter.getColumnCapabilities(columnName).getType()); + rowSignatureBuilder.add(columnName, adapter.getColumnCapabilities(columnName).toColumnType()); } // Perform test diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java index 0b1004e30b57..e10d64241f2d 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.SimpleAscendingOffset; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts; @@ -46,7 +46,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes @Test public void testDefaultType() { - Assert.assertEquals(ValueType.COMPLEX, forSelector(null).defaultType()); + Assert.assertEquals(ColumnType.UNKNOWN_COMPLEX, forSelector(null).defaultType()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java index 33199b328644..fed718de5299 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java @@ -29,7 +29,7 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; @@ -267,7 +267,7 @@ protected VirtualColumn makeExpressionVirtualColumn(String expression, String co return new ExpressionVirtualColumn( columnName, expression, - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ); } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java index 694edde1e9de..813db7d4cda7 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinFilterAnalyzerTest.java @@ -36,7 +36,7 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.filter.BoundFilter; import org.apache.druid.segment.filter.FalseFilter; @@ -405,7 +405,7 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC new ExpressionVirtualColumn( "v1", "concat('virtual-column-', \"channel\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -476,7 +476,7 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu new ExpressionVirtualColumn( "v0", "upper(\"r1.regionName\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ) @@ -763,7 +763,7 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", "reverse(countryIsoCode)", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ); Assert.assertEquals( @@ -997,7 +997,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", "concat(countryIsoCode, regionIsoCode)", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -1087,7 +1087,7 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-0", "concat(countryIsoCode, regionIsoCode)", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -2376,13 +2376,13 @@ public boolean supportsRequiredColumnRewrite() new ExpressionVirtualColumn( rewrittenRegionIsoCodeColumnName, "(upper [(lower [regionIsoCode])])", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ), new ExpressionVirtualColumn( rewrittenCountryIsoCodeColumnName, "(upper [(lower [countryIsoCode])])", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 820ab79d8da0..5535ff08e053 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -53,8 +53,8 @@ import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnConfig; +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.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.table.RowBasedIndexedTable; import org.junit.Assert; @@ -90,26 +90,26 @@ public class JoinTestHelper ); private static final RowSignature COUNTRIES_SIGNATURE = RowSignature.builder() - .add("countryNumber", ValueType.LONG) - .add("countryIsoCode", ValueType.STRING) - .add("countryName", ValueType.STRING) + .add("countryNumber", ColumnType.LONG) + .add("countryIsoCode", ColumnType.STRING) + .add("countryName", ColumnType.STRING) .build(); private static final RowSignature REGIONS_SIGNATURE = RowSignature.builder() - .add("regionIsoCode", ValueType.STRING) - .add("countryIsoCode", ValueType.STRING) - .add("regionName", ValueType.STRING) - .add("extraField", ValueType.STRING) + .add("regionIsoCode", ColumnType.STRING) + .add("countryIsoCode", ColumnType.STRING) + .add("regionName", ColumnType.STRING) + .add("extraField", ColumnType.STRING) .build(); private static final ColumnProcessorFactory> SIMPLE_READER = new ColumnProcessorFactory>() { @Override - public ValueType defaultType() + public ColumnType defaultType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override @@ -159,7 +159,7 @@ public ToLongFunction> timestampFunction() @Override public Function, Object> columnFunction(String columnName) { - final ValueType columnType = signature.getColumnType(columnName).orElse(null); + final ColumnType columnType = signature.getColumnType(columnName).orElse(null); if (columnType == null) { return row -> row.get(columnName); diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java index 6a057e58ebff..6d6e930fb88c 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java @@ -32,7 +32,7 @@ import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -477,9 +477,9 @@ private static IndexedTable.Index stringToLengthIndex() return new IndexedTable.Index() { @Override - public ValueType keyType() + public ColumnType keyType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override @@ -507,9 +507,9 @@ private static IndexedTable.Index certainStringToThreeIndex() return new IndexedTable.Index() { @Override - public ValueType keyType() + public ColumnType keyType() { - return ValueType.STRING; + return ColumnType.STRING; } @Override @@ -541,9 +541,9 @@ private static IndexedTable.Index longPlusOneIndex() return new IndexedTable.Index() { @Override - public ValueType keyType() + public ColumnType keyType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override @@ -577,9 +577,9 @@ private static IndexedTable.Index longAlwaysOneTwoThreeIndex() return new IndexedTable.Index() { @Override - public ValueType keyType() + public ColumnType keyType() { - return ValueType.LONG; + return ColumnType.LONG; } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java index a9b1ae599d88..404ded8d44ac 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.ConstantDimensionSelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.join.JoinConditionAnalysis; @@ -90,9 +91,9 @@ public ColumnCapabilities getColumnCapabilities(String columnName) new Object[]{"baz", null, 1L} ), RowSignature.builder() - .add(KEY_COLUMN, ValueType.STRING) - .add(VALUE_COLUMN, ValueType.LONG) - .add(ALL_SAME_COLUMN, ValueType.LONG) + .add(KEY_COLUMN, ColumnType.STRING) + .add(VALUE_COLUMN, ColumnType.LONG) + .add(ALL_SAME_COLUMN, ColumnType.LONG) .build() ); diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexBuilderTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexBuilderTest.java index 5925c5f34328..e37efae2deac 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexBuilderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexBuilderTest.java @@ -21,7 +21,7 @@ import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -37,7 +37,7 @@ public class RowBasedIndexBuilderTest public void test_stringKey_uniqueKeys() { final RowBasedIndexBuilder builder = - new RowBasedIndexBuilder(ValueType.STRING) + new RowBasedIndexBuilder(ColumnType.STRING) .add("abc") .add("") .add(null) @@ -47,7 +47,7 @@ public void test_stringKey_uniqueKeys() final IndexedTable.Index index = builder.build(); Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class)); - Assert.assertEquals(ValueType.STRING, index.keyType()); + Assert.assertEquals(ColumnType.STRING, index.keyType()); Assert.assertTrue(index.areKeysUnique()); Assert.assertEquals(intList(0), index.find("abc")); @@ -66,7 +66,7 @@ public void test_stringKey_uniqueKeys() public void test_stringKey_duplicateKeys() { final RowBasedIndexBuilder builder = - new RowBasedIndexBuilder(ValueType.STRING) + new RowBasedIndexBuilder(ColumnType.STRING) .add("abc") .add("") .add(null) @@ -77,7 +77,7 @@ public void test_stringKey_duplicateKeys() final IndexedTable.Index index = builder.build(); Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class)); - Assert.assertEquals(ValueType.STRING, index.keyType()); + Assert.assertEquals(ColumnType.STRING, index.keyType()); Assert.assertFalse(index.areKeysUnique()); Assert.assertEquals(intList(0, 3), index.find("abc")); @@ -96,7 +96,7 @@ public void test_stringKey_duplicateKeys() public void test_longKey_uniqueKeys() { final RowBasedIndexBuilder builder = - new RowBasedIndexBuilder(ValueType.LONG) + new RowBasedIndexBuilder(ColumnType.LONG) .add(1) .add(5) .add(2); @@ -104,7 +104,7 @@ public void test_longKey_uniqueKeys() final IndexedTable.Index index = builder.build(); Assert.assertThat(index, CoreMatchers.instanceOf(UniqueLongArrayIndex.class)); - Assert.assertEquals(ValueType.LONG, index.keyType()); + Assert.assertEquals(ColumnType.LONG, index.keyType()); Assert.assertTrue(index.areKeysUnique()); Assert.assertEquals(intList(0), index.find(1L)); @@ -122,7 +122,7 @@ public void test_longKey_uniqueKeys() public void test_longKey_uniqueKeys_farApart() { final RowBasedIndexBuilder builder = - new RowBasedIndexBuilder(ValueType.LONG) + new RowBasedIndexBuilder(ColumnType.LONG) .add(1) .add(10_000_000) .add(2); @@ -130,7 +130,7 @@ public void test_longKey_uniqueKeys_farApart() final IndexedTable.Index index = builder.build(); Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class)); - Assert.assertEquals(ValueType.LONG, index.keyType()); + Assert.assertEquals(ColumnType.LONG, index.keyType()); Assert.assertTrue(index.areKeysUnique()); Assert.assertEquals(intList(0), index.find(1L)); @@ -148,7 +148,7 @@ public void test_longKey_uniqueKeys_farApart() public void test_longKey_duplicateKeys() { final RowBasedIndexBuilder builder = - new RowBasedIndexBuilder(ValueType.LONG) + new RowBasedIndexBuilder(ColumnType.LONG) .add(1) .add(5) .add(1) @@ -157,7 +157,7 @@ public void test_longKey_duplicateKeys() final IndexedTable.Index index = builder.build(); Assert.assertThat(index, CoreMatchers.instanceOf(MapIndex.class)); - Assert.assertEquals(ValueType.LONG, index.keyType()); + Assert.assertEquals(ColumnType.LONG, index.keyType()); Assert.assertFalse(index.areKeysUnique()); Assert.assertEquals(intList(0, 2), index.find("1")); diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java index 371457f42458..2821b749d91f 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java @@ -22,8 +22,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +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.join.JoinTestHelper; import org.junit.Assert; import org.junit.Before; @@ -73,9 +73,9 @@ public void test_rowSignature_countries() { Assert.assertEquals( RowSignature.builder() - .add("countryNumber", ValueType.LONG) - .add("countryIsoCode", ValueType.STRING) - .add("countryName", ValueType.STRING) + .add("countryNumber", ColumnType.LONG) + .add("countryIsoCode", ColumnType.STRING) + .add("countryName", ColumnType.STRING) .build(), countriesTable.rowSignature() ); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java index b41342486750..8323f4341a8e 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java @@ -37,8 +37,8 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.StringDictionaryEncodedColumn; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ReadableOffset; @@ -183,7 +183,7 @@ public ColumnValueSelector makeColumnValueSelector( @Override public ColumnCapabilities capabilities(String columnName) { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING) + ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true); if (enableBitmaps) { capabilities.setHasBitmapIndexes(true); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java index c6d273e3d894..a28faac94c03 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionPlannerTest.java @@ -21,12 +21,13 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.Parser; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -45,27 +46,27 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest ImmutableMap.builder() .put( "long1", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG) ) .put( "long2", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG) ) .put( "float1", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT) ) .put( "float2", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT) ) .put( "double1", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE) ) .put( "double2", - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE) + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE) ) .put( "scalar_string", @@ -74,7 +75,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest .put( // segment style single value dictionary encoded with unique sorted dictionary "scalar_dictionary_string", - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) .setHasBitmapIndexes(true) .setDictionaryValuesSorted(true) @@ -84,7 +85,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest .put( // dictionary encoded but not unique or sorted, maybe an indexed table from a join result "scalar_dictionary_string_nonunique", - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) .setHasBitmapIndexes(false) .setDictionaryValuesSorted(false) @@ -94,12 +95,12 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest .put( // string with unknown multi-valuedness "string_unknown", - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) ) .put( // dictionary encoded multi valued string dimension "multi_dictionary_string", - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(true) .setHasBitmapIndexes(true) .setDictionaryValuesUnique(true) @@ -109,7 +110,7 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest .put( // simple multi valued string dimension unsorted "multi_dictionary_string_nonunique", - new ColumnCapabilitiesImpl().setType(ValueType.STRING) + new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setDictionaryEncoded(false) .setHasBitmapIndexes(false) .setDictionaryValuesUnique(false) @@ -118,27 +119,27 @@ public class ExpressionPlannerTest extends InitializedNullHandlingTest ) .put( "string_array_1", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.STRING_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.STRING_ARRAY) ) .put( "string_array_2", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.STRING_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.STRING_ARRAY) ) .put( "long_array_1", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.LONG_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.LONG_ARRAY) ) .put( "long_array_2", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.LONG_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.LONG_ARRAY) ) .put( "double_array_1", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.DOUBLE_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.DOUBLE_ARRAY) ) .put( "double_array_2", - ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ValueType.DOUBLE_ARRAY) + ColumnCapabilitiesImpl.createSimpleArrayColumnCapabilities(ColumnType.DOUBLE_ARRAY) ) .build(); @@ -182,11 +183,10 @@ public void testUnknown() // this expression has no "unapplied bindings", nothing to apply Assert.assertEquals("concat(\"x\", 'x')", thePlan.getAppliedExpression().stringify()); Assert.assertEquals("concat(\"x\", 'x')", thePlan.getAppliedFoldExpression("__acc").stringify()); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); ColumnCapabilities inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -243,11 +243,10 @@ public void testScalarStringNondictionaryEncoded() ); Assert.assertEquals("concat(\"scalar_string\", 'x')", thePlan.getAppliedExpression().stringify()); Assert.assertEquals("concat(\"scalar_string\", 'x')", thePlan.getAppliedFoldExpression("__acc").stringify()); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); ColumnCapabilities inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -280,11 +279,10 @@ public void testScalarNumeric() Assert.assertEquals("(\"long1\" + 5)", thePlan.getAppliedExpression().stringify()); Assert.assertEquals("(\"long1\" + 5)", thePlan.getAppliedFoldExpression("__acc").stringify()); Assert.assertEquals("(\"long1\" + 5)", thePlan.getAppliedFoldExpression("long1").stringify()); - Assert.assertEquals(ExprType.LONG, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.LONG, thePlan.getOutputType()); ColumnCapabilities inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.LONG, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); if (NullHandling.sqlCompatible()) { Assert.assertTrue(inferred.hasNulls().isMaybeTrue()); } else { @@ -298,7 +296,7 @@ public void testScalarNumeric() Assert.assertFalse(inferred.hasSpatialIndexes()); thePlan = plan("long1 + 5.0"); - Assert.assertEquals(ExprType.DOUBLE, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.DOUBLE, thePlan.getOutputType()); thePlan = plan("double1 * double2"); Assert.assertTrue( @@ -320,11 +318,10 @@ public void testScalarNumeric() Assert.assertEquals("(\"double1\" * \"double2\")", thePlan.getAppliedExpression().stringify()); Assert.assertEquals("(\"double1\" * \"double2\")", thePlan.getAppliedFoldExpression("__acc").stringify()); Assert.assertEquals("(\"double1\" * \"double2\")", thePlan.getAppliedFoldExpression("double1").stringify()); - Assert.assertEquals(ExprType.DOUBLE, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.DOUBLE, thePlan.getOutputType()); inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.DOUBLE, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); if (NullHandling.sqlCompatible()) { Assert.assertTrue(inferred.hasNulls().isMaybeTrue()); } else { @@ -363,11 +360,10 @@ public void testScalarStringDictionaryEncoded() "concat(\"scalar_dictionary_string\", 'x')", thePlan.getAppliedFoldExpression("__acc").stringify() ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); ColumnCapabilities inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isTrue()); Assert.assertTrue(inferred.isDictionaryEncoded().isTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -407,11 +403,10 @@ public void testScalarStringDictionaryEncoded() "concat(\"scalar_dictionary_string\", \"scalar_dictionary_string_nonunique\")", thePlan.getAppliedFoldExpression("scalar_dictionary_string_nonunique").stringify() ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -459,11 +454,10 @@ public void testMultiValueStringDictionaryEncoded() ExpressionPlan.Trait.VECTORIZABLE ) ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); ColumnCapabilities inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isMaybeTrue()); Assert.assertTrue(inferred.isDictionaryEncoded().isTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -495,7 +489,7 @@ public void testMultiValueStringDictionaryEncoded() "fold((\"multi_dictionary_string_nonunique\", \"scalar_string\") -> concat(\"scalar_string\", \"multi_dictionary_string_nonunique\"), \"multi_dictionary_string_nonunique\", \"scalar_string\")", thePlan.getAppliedFoldExpression("scalar_string").stringify() ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); inferred = thePlan.inferColumnCapabilities(null); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); @@ -516,7 +510,7 @@ public void testMultiValueStringDictionaryEncoded() ExpressionPlan.Trait.VECTORIZABLE ) ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); // whoa Assert.assertEquals( "cartesian_map((\"multi_dictionary_string\", \"multi_dictionary_string_nonunique\") -> concat(\"multi_dictionary_string\", \"multi_dictionary_string_nonunique\"), \"multi_dictionary_string\", \"multi_dictionary_string_nonunique\")", @@ -572,7 +566,7 @@ public void testMultiValueStringDictionaryEncodedIllegalAccumulator() ExpressionPlan.Trait.VECTORIZABLE ) ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); thePlan = plan("concat(multi_dictionary_string, multi_dictionary_string_nonunique)"); Assert.assertTrue( @@ -592,7 +586,7 @@ public void testMultiValueStringDictionaryEncodedIllegalAccumulator() // what happens if we try to use a multi-valued input that was not explicitly used as multi-valued as the // accumulator? thePlan.getAppliedFoldExpression("multi_dictionary_string"); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); } @Test @@ -632,10 +626,9 @@ public void testArrayOutput() ExpressionPlan thePlan = plan("array_append(scalar_string, 'x')"); assertArrayInAndOut(thePlan); // with a string hint, it should look like a multi-valued string - ColumnCapabilities inferred = thePlan.inferColumnCapabilities(ValueType.STRING); + ColumnCapabilities inferred = thePlan.inferColumnCapabilities(ColumnType.STRING); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isMaybeTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -644,10 +637,9 @@ public void testArrayOutput() Assert.assertFalse(inferred.hasBitmapIndexes()); Assert.assertFalse(inferred.hasSpatialIndexes()); // with no hint though, let the array free - inferred = thePlan.inferColumnCapabilities(ValueType.STRING_ARRAY); + inferred = thePlan.inferColumnCapabilities(ColumnType.STRING_ARRAY); Assert.assertNotNull(inferred); - Assert.assertEquals(ValueType.STRING_ARRAY, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); + Assert.assertEquals(ColumnType.STRING_ARRAY, inferred.toColumnType()); Assert.assertTrue(inferred.hasNulls().isMaybeTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -658,7 +650,7 @@ public void testArrayOutput() Assert.assertEquals("array_append(\"scalar_string\", 'x')", thePlan.getAppliedExpression().stringify()); Assert.assertEquals("array_append(\"scalar_string\", 'x')", thePlan.getAppliedFoldExpression("__acc").stringify()); - Assert.assertEquals(ExprType.STRING_ARRAY, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, thePlan.getOutputType()); // multi-valued are cool too thePlan = plan("array_append(multi_dictionary_string, 'x')"); @@ -667,7 +659,7 @@ public void testArrayOutput() // what about incomplete inputs with arrays? they are not reported as incomplete because they are treated as arrays thePlan = plan("array_append(string_unknown, 'x')"); assertArrayInAndOut(thePlan); - Assert.assertEquals(ExprType.STRING_ARRAY, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, thePlan.getOutputType()); // what about if it is the scalar argument? there it is thePlan = plan("array_append(multi_dictionary_string, string_unknown)"); @@ -704,10 +696,9 @@ public void testScalarOutputMultiValueInput() { ExpressionPlan thePlan = plan("array_to_string(array_append(scalar_string, 'x'), ',')"); assertArrayInput(thePlan); - ColumnCapabilities inferred = thePlan.inferColumnCapabilities(ValueType.STRING); + ColumnCapabilities inferred = thePlan.inferColumnCapabilities(ColumnType.STRING); Assert.assertNotNull(inferred); Assert.assertEquals(ValueType.STRING, inferred.getType()); - Assert.assertNull(inferred.getComplexTypeName()); Assert.assertTrue(inferred.hasNulls().isTrue()); Assert.assertFalse(inferred.isDictionaryEncoded().isMaybeTrue()); Assert.assertFalse(inferred.areDictionaryValuesSorted().isMaybeTrue()); @@ -724,7 +715,7 @@ public void testScalarOutputMultiValueInput() "array_to_string(array_append(\"scalar_string\", 'x'), ',')", thePlan.getAppliedFoldExpression("__acc").stringify() ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); // what about a multi-valued input thePlan = plan("array_to_string(array_append(scalar_string, multi_dictionary_string), ',')"); @@ -739,7 +730,7 @@ public void testScalarOutputMultiValueInput() thePlan.getAppliedFoldExpression("scalar_string").stringify() ); // why is this null - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); } @Test @@ -756,12 +747,12 @@ public void testScalarOutputArrayInput() "array_to_string(array_append(\"string_array_1\", 'x'), ',')", thePlan.getAppliedFoldExpression("__acc").stringify() ); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); thePlan = plan("array_to_string(array_concat(string_array_1, string_array_2), ',')"); assertArrayInput(thePlan); - Assert.assertEquals(ExprType.STRING, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING, thePlan.getOutputType()); thePlan = plan("fold((x, acc) -> acc + x, array_concat(long_array_1, long_array_2), 0)"); assertArrayInput(thePlan); @@ -773,7 +764,7 @@ public void testScalarOutputArrayInput() "fold((\"x\", \"acc\") -> (\"acc\" + \"x\"), array_concat(\"long_array_1\", \"long_array_2\"), 0)", thePlan.getAppliedFoldExpression("__acc").stringify() ); - Assert.assertEquals(ExprType.LONG, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.LONG, thePlan.getOutputType()); thePlan = plan("fold((x, acc) -> acc * x, array_concat(double_array_1, double_array_2), 0.0)"); assertArrayInput(thePlan); @@ -785,7 +776,7 @@ public void testScalarOutputArrayInput() "fold((\"x\", \"acc\") -> (\"acc\" * \"x\"), array_concat(\"double_array_1\", \"double_array_2\"), 0.0)", thePlan.getAppliedFoldExpression("__acc").stringify() ); - Assert.assertEquals(ExprType.DOUBLE, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.DOUBLE, thePlan.getOutputType()); } @Test @@ -808,12 +799,12 @@ public void testArrayConstruction() ExpressionPlan.Trait.VECTORIZABLE ) ); - Assert.assertEquals(ExprType.LONG_ARRAY, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.LONG_ARRAY, thePlan.getOutputType()); thePlan = plan("array(long1, double1)"); - Assert.assertEquals(ExprType.DOUBLE_ARRAY, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.DOUBLE_ARRAY, thePlan.getOutputType()); thePlan = plan("array(long1, double1, scalar_string)"); - Assert.assertEquals(ExprType.STRING_ARRAY, thePlan.getOutputType()); + Assert.assertEquals(ExpressionType.STRING_ARRAY, thePlan.getOutputType()); } private static ExpressionPlan plan(String expression) diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java index 40955f69b3a6..e940e68fc123 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVectorSelectorsTest.java @@ -26,7 +26,7 @@ import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.Expr; 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.Parser; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; @@ -133,7 +133,7 @@ public static Iterable constructorFeeder() return EXPRESSIONS.stream().map(x -> new Object[]{x}).collect(Collectors.toList()); } - private ExprType outputType; + private ExpressionType outputType; private String expression; public ExpressionVectorSelectorsTest(String expression) @@ -157,7 +157,7 @@ public ColumnCapabilities getColumnCapabilities(String column) } ); if (outputType == null) { - outputType = ExprType.STRING; + outputType = ExpressionType.STRING; } } @@ -169,7 +169,7 @@ public void sanityTestVectorizedExpressionSelector() public static void sanityTestVectorizedExpressionSelectors( String expression, - @Nullable ExprType outputType, + @Nullable ExpressionType outputType, QueryableIndex index, Closer closer, int rowsPerSegment @@ -181,7 +181,7 @@ public static void sanityTestVectorizedExpressionSelectors( new ExpressionVirtualColumn( "v", expression, - ExprType.toValueType(outputType), + ExpressionType.toColumnType(outputType), TestExprMacroTable.INSTANCE ) ) @@ -220,7 +220,7 @@ public static void sanityTestVectorizedExpressionSelectors( } while (!cursor.isDone()) { boolean[] nulls; - switch (outputType) { + switch (outputType.getType()) { case LONG: nulls = selector.getNullVector(); long[] longs = selector.getLongVector(); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index 93c86fd3db37..a5dac6716867 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; @@ -118,83 +119,83 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest private static final ExpressionVirtualColumn X_PLUS_Y = new ExpressionVirtualColumn( "expr", "x + y", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn CONSTANT_LIKE = new ExpressionVirtualColumn( "expr", "like('foo', 'f%')", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn CONSTANT_NULL_ARITHMETIC = new ExpressionVirtualColumn( "expr", "2.1 + null", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn Z_LIKE = new ExpressionVirtualColumn( "expr", "like(z, 'f%')", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn Z_CONCAT_X = new ExpressionVirtualColumn( "expr", "z + cast(x, 'string')", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn Z_CONCAT_NONEXISTENT = new ExpressionVirtualColumn( "expr", "concat(z, nonexistent)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn TIME_FLOOR = new ExpressionVirtualColumn( "expr", "timestamp_floor(__time, 'P1D')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_LONG = new ExpressionVirtualColumn( "expr", "x * 2", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_FLOAT = new ExpressionVirtualColumn( "expr", "x * 2", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_LIST_IMPLICIT = new ExpressionVirtualColumn( "expr", "b * 2", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_LIST_EXPLICIT = new ExpressionVirtualColumn( "expr", "map(b -> b * 2, b)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_LIST_SELF_IMPLICIT = new ExpressionVirtualColumn( "expr", "b * b", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); private static final ExpressionVirtualColumn SCALE_LIST_SELF_EXPLICIT = new ExpressionVirtualColumn( "expr", "map(b -> b * b, b)", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ); @@ -355,7 +356,7 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) @Override public ColumnCapabilities getColumnCapabilities(String column) { - return new ColumnCapabilitiesImpl().setType(ValueType.STRING) + return new ColumnCapabilitiesImpl().setType(ColumnType.STRING) .setHasMultipleValues(true) .setDictionaryEncoded(true); } @@ -742,7 +743,7 @@ public void testExprEvalSelectorWithLongsAndNulls() RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), CURRENT_ROW::get, - RowSignature.builder().add("x", ValueType.LONG).build(), + RowSignature.builder().add("x", ColumnType.LONG).build(), false ), Parser.parse(SCALE_LONG.getExpression(), TestExprMacroTable.INSTANCE) @@ -765,7 +766,7 @@ public void testExprEvalSelectorWithDoublesAndNulls() RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), CURRENT_ROW::get, - RowSignature.builder().add("x", ValueType.DOUBLE).build(), + RowSignature.builder().add("x", ColumnType.DOUBLE).build(), false ), Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE) @@ -788,7 +789,7 @@ public void testExprEvalSelectorWithFloatAndNulls() RowBasedColumnSelectorFactory.create( RowAdapters.standardRow(), CURRENT_ROW::get, - RowSignature.builder().add("x", ValueType.FLOAT).build(), + RowSignature.builder().add("x", ColumnType.FLOAT).build(), false ), Parser.parse(SCALE_FLOAT.getExpression(), TestExprMacroTable.INSTANCE) diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java index cc7f5e4f05e0..9dbc25d4317e 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.column.BitmapIndex; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +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.filter.SelectorFilter; @@ -64,7 +65,7 @@ public void testListFilteredVirtualColumnNilDimensionSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( ALLOW_VIRTUAL_NAME, - new DefaultDimensionSpec(NON_EXISTENT_COLUMN_NAME, NON_EXISTENT_COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(NON_EXISTENT_COLUMN_NAME, NON_EXISTENT_COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), true ); @@ -79,7 +80,7 @@ public void testListFilteredVirtualColumnNilColumnValueSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( ALLOW_VIRTUAL_NAME, - new DefaultDimensionSpec(NON_EXISTENT_COLUMN_NAME, NON_EXISTENT_COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(NON_EXISTENT_COLUMN_NAME, NON_EXISTENT_COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), true ); @@ -95,7 +96,7 @@ public void testListFilteredVirtualColumnAllowListDimensionSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( ALLOW_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), true ); @@ -111,7 +112,7 @@ public void testListFilteredVirtualColumnAllowListColumnValueSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( ALLOW_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), true ); @@ -127,7 +128,7 @@ public void testListFilteredVirtualColumnDenyListDimensionSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( DENY_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), false ); @@ -143,7 +144,7 @@ public void testListFilteredVirtualColumnDenyListColumnValueSelector() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( DENY_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), false ); @@ -159,7 +160,7 @@ public void testFilterListFilteredVirtualColumnAllowIndex() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( ALLOW_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("b", "c"), true ); @@ -213,7 +214,7 @@ public void testFilterListFilteredVirtualColumnDenyIndex() { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( DENY_VIRTUAL_NAME, - new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ValueType.STRING), + new DefaultDimensionSpec(COLUMN_NAME, COLUMN_NAME, ColumnType.STRING), ImmutableSet.of("a", "b"), false ); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnTest.java index 7238ac2da4c1..b36e958ad475 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnTest.java @@ -25,7 +25,7 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; @@ -38,7 +38,7 @@ public void testSerdeAllowList() throws JsonProcessingException { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( "hello", - new DefaultDimensionSpec("column", "output", ValueType.STRING), + new DefaultDimensionSpec("column", "output", ColumnType.STRING), ImmutableSet.of("foo", "bar"), true ); @@ -52,7 +52,7 @@ public void testSerdeDenyList() throws JsonProcessingException { ListFilteredVirtualColumn virtualColumn = new ListFilteredVirtualColumn( "hello", - new DefaultDimensionSpec("column", "output", ValueType.STRING), + new DefaultDimensionSpec("column", "output", ColumnType.STRING), ImmutableSet.of("foo", "bar"), false ); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java index 850c4d52ad0e..126e304ae04a 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VectorizedVirtualColumnTest.java @@ -44,7 +44,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.timeline.SegmentId; import org.junit.Before; import org.junit.Rule; @@ -120,7 +120,7 @@ public void setup() public void testGroupBySingleValueString() { testGroupBy(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false) @@ -133,7 +133,7 @@ public void testGroupByMultiValueString() // cannot currently group by string columns that might be multi valued cannotVectorize(); testGroupBy(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(true) @@ -146,7 +146,7 @@ public void testGroupByMultiValueStringUnknown() // cannot currently group by string columns that might be multi valued cannotVectorize(); testGroupBy(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) ); @@ -156,7 +156,7 @@ public void testGroupByMultiValueStringUnknown() public void testGroupBySingleValueStringNotDictionaryEncoded() { testGroupBy(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(false) .setDictionaryValuesUnique(false) .setHasMultipleValues(false) @@ -169,7 +169,7 @@ public void testGroupByMultiValueStringNotDictionaryEncoded() // cannot currently group by string columns that might be multi valued cannotVectorize(); testGroupBy(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(false) .setDictionaryValuesUnique(false) .setHasMultipleValues(true) @@ -179,26 +179,26 @@ public void testGroupByMultiValueStringNotDictionaryEncoded() @Test public void testGroupByLong() { - testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG)); + testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG)); } @Test public void testGroupByDouble() { - testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE)); + testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE)); } @Test public void testGroupByFloat() { - testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT)); + testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT)); } @Test public void testTimeseriesSingleValueString() { testTimeseries(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false) @@ -209,7 +209,7 @@ public void testTimeseriesSingleValueString() public void testTimeseriesMultiValueString() { testTimeseries(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(true) @@ -220,7 +220,7 @@ public void testTimeseriesMultiValueString() public void testTimeseriesMultiValueStringUnknown() { testTimeseries(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) ); @@ -230,7 +230,7 @@ public void testTimeseriesMultiValueStringUnknown() public void testTimeseriesSingleValueStringNotDictionaryEncoded() { testTimeseries(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(false) .setDictionaryValuesUnique(false) .setHasMultipleValues(false) @@ -241,7 +241,7 @@ public void testTimeseriesSingleValueStringNotDictionaryEncoded() public void testTimeseriesMultiValueStringNotDictionaryEncoded() { testTimeseries(new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(false) .setDictionaryValuesUnique(false) .setHasMultipleValues(true) @@ -251,19 +251,19 @@ public void testTimeseriesMultiValueStringNotDictionaryEncoded() @Test public void testTimeseriesLong() { - testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG)); + testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG)); } @Test public void testTimeseriesDouble() { - testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE)); + testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.DOUBLE)); } @Test public void testTimeseriesFloat() { - testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT)); + testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT)); } @Test @@ -271,7 +271,7 @@ public void testTimeseriesForceContextCannotVectorize() { cannotVectorize(); testTimeseries( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_VECTORIZE_FORCE, false ); @@ -282,7 +282,7 @@ public void testTimeseriesForceVirtualContextCannotVectorize() { cannotVectorize(); testTimeseries( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE, false ); @@ -293,7 +293,7 @@ public void testTimeseriesTrueVirtualContextCannotVectorize() { expectNonvectorized(); testTimeseries( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_USE_DEFAULTS, true ); @@ -304,7 +304,7 @@ public void testTimeseriesContradictionVectorizeFalseVirtualForce() { expectNonvectorized(); testTimeseries( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE, true ); @@ -315,7 +315,7 @@ public void testTimeseriesContradictionVectorizeForceVirtualFalse() { cannotVectorize(); testTimeseries( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE, true ); @@ -325,7 +325,7 @@ public void testTimeseriesContradictionVectorizeForceVirtualFalse() public void testTimeseriesContradictionVectorizeFalseVirtualForceNoVirtualColumns() { testTimeseriesNoVirtual( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE ); } @@ -333,7 +333,7 @@ public void testTimeseriesContradictionVectorizeFalseVirtualForceNoVirtualColumn public void testTimeseriesContradictionVectorizeForceVirtualFalseNoVirtual() { testTimeseriesNoVirtual( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE ); } @@ -342,7 +342,7 @@ public void testTimeseriesContradictionVectorizeForceVirtualFalseNoVirtual() public void testTimeseriesForceDoestAffectWhenNoVirtualColumns() { testTimeseriesNoVirtual( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_VECTORIZE_TRUE_VIRTUAL_FORCE ); } @@ -352,7 +352,7 @@ public void testGroupByForceContextCannotVectorize() { cannotVectorize(); testGroupBy( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_VECTORIZE_FORCE, false ); @@ -364,7 +364,7 @@ public void testGroupByForceVirtualContextCannotVectorize() cannotVectorize(); testGroupBy( new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false), @@ -378,7 +378,7 @@ public void testGroupByTrueVirtualContextCannotVectorize() { expectNonvectorized(); testGroupBy( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_USE_DEFAULTS, false ); @@ -389,7 +389,7 @@ public void testGroupByContradictionVectorizeFalseVirtualForce() { expectNonvectorized(); testGroupBy( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FALSE_VIRTUAL_FORCE, true ); @@ -400,7 +400,7 @@ public void testGroupByContradictionVectorizeForceVirtualFalse() { cannotVectorize(); testGroupBy( - ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT), + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.FLOAT), CONTEXT_CONTRADICTION_VECTORIZE_FORCE_VIRTUAL_FALSE, true ); @@ -411,7 +411,7 @@ public void testGroupByContradictionVectorizeFalseVirtualForceNoVirtual() { testGroupByNoVirtual( new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false), @@ -424,7 +424,7 @@ public void testGroupByContradictionVectorizeForceVirtualFalseNoVirtual() { testGroupByNoVirtual( new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false), @@ -437,7 +437,7 @@ public void testGroupByForceDoestAffectWhenNoVirtualColumns() { testGroupByNoVirtual( new ColumnCapabilitiesImpl() - .setType(ValueType.STRING) + .setType(ColumnType.STRING) .setDictionaryEncoded(true) .setDictionaryValuesUnique(true) .setHasMultipleValues(false), @@ -534,7 +534,7 @@ private void testGroupBy(ColumnCapabilities capabilities, Map co .setVirtualColumns( new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities, canVectorize) ) - .addDimension(new DefaultDimensionSpec(ALWAYS_TWO, ALWAYS_TWO, capabilities.getType())) + .addDimension(new DefaultDimensionSpec(ALWAYS_TWO, ALWAYS_TWO, capabilities.toColumnType())) .setAggregatorSpecs(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO)) .setInterval("2000/2030") .setContext(context) @@ -563,7 +563,7 @@ private void testGroupByNoVirtual(ColumnCapabilities capabilities, Map theColumns = ImmutableList.of( - new ExpressionVirtualColumn("expr", "x + y", ValueType.FLOAT, TestExprMacroTable.INSTANCE), - new ExpressionVirtualColumn("expr2", "x + z", ValueType.FLOAT, TestExprMacroTable.INSTANCE) + new ExpressionVirtualColumn("expr", "x + y", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("expr2", "x + z", ColumnType.FLOAT, TestExprMacroTable.INSTANCE) ); final VirtualColumns virtualColumns = VirtualColumns.create(theColumns); @@ -283,7 +283,7 @@ private VirtualColumns makeVirtualColumns() final ExpressionVirtualColumn expr = new ExpressionVirtualColumn( "expr", "1", - ValueType.FLOAT, + ColumnType.FLOAT, TestExprMacroTable.INSTANCE ); final DottyVirtualColumn dotty = new DottyVirtualColumn("foo"); @@ -417,7 +417,7 @@ public boolean isNull() @Override public ColumnCapabilities capabilities(String columnName) { - return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ColumnType.LONG); } @Override diff --git a/server/src/test/java/org/apache/druid/query/lookup/LookupSerdeModuleTest.java b/server/src/test/java/org/apache/druid/query/lookup/LookupSerdeModuleTest.java index 4329fe1115c6..5ef82e6ecb42 100644 --- a/server/src/test/java/org/apache/druid/query/lookup/LookupSerdeModuleTest.java +++ b/server/src/test/java/org/apache/druid/query/lookup/LookupSerdeModuleTest.java @@ -34,7 +34,7 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.ExpressionTransform; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.junit.Assert; @@ -73,7 +73,7 @@ public void testExpressionVirtualColumnSerde() throws Exception final ExpressionVirtualColumn virtualColumn = new ExpressionVirtualColumn( "v", "lookup(xxx, 'beep')", - ValueType.STRING, + ColumnType.STRING, injector.getInstance(ExprMacroTable.class) ); diff --git a/server/src/test/java/org/apache/druid/segment/InlineSegmentWranglerTest.java b/server/src/test/java/org/apache/druid/segment/InlineSegmentWranglerTest.java index fb0bfa1a5280..80bd3eb2eb3b 100644 --- a/server/src/test/java/org/apache/druid/segment/InlineSegmentWranglerTest.java +++ b/server/src/test/java/org/apache/druid/segment/InlineSegmentWranglerTest.java @@ -24,8 +24,8 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.TableDataSource; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.ValueType; import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -46,7 +46,7 @@ public class InlineSegmentWranglerTest new Object[]{"foo", 1L}, new Object[]{"bar", 2L} ), - RowSignature.builder().add("str", ValueType.STRING).add("long", ValueType.LONG).build() + RowSignature.builder().add("str", ColumnType.STRING).add("long", ColumnType.LONG).build() ); @Test diff --git a/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java b/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java index 2a5bf3e5c7ce..07e36a49b3da 100644 --- a/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java @@ -23,8 +23,8 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.TableDataSource; +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.join.table.IndexedTableJoinable; import org.hamcrest.CoreMatchers; import org.junit.Assert; @@ -48,7 +48,7 @@ public class InlineJoinableFactoryTest new Object[]{"foo", 1L}, new Object[]{"bar", 2L} ), - RowSignature.builder().add("str", ValueType.STRING).add("long", ValueType.LONG).build() + RowSignature.builder().add("str", ColumnType.STRING).add("long", ColumnType.LONG).build() ); @Test diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index 7c47377b9ce0..cf62a1400664 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -70,8 +70,8 @@ import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.SegmentWrangler; 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.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; @@ -136,8 +136,8 @@ public class ClientQuerySegmentWalkerTest .build(), RowSignature.builder() .addTimeColumn() - .add("s", ValueType.STRING) - .add("n", ValueType.LONG) + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) .build() ); @@ -150,8 +150,8 @@ public class ClientQuerySegmentWalkerTest .build(), RowSignature.builder() .addTimeColumn() - .add("s", ValueType.STRING) - .add("n", ValueType.LONG) + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) .build() ); @@ -164,8 +164,8 @@ public class ClientQuerySegmentWalkerTest .build(), RowSignature.builder() .addTimeColumn() - .add("s", ValueType.STRING) - .add("n", ValueType.LONG) + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) .build() ); @@ -182,11 +182,11 @@ public class ClientQuerySegmentWalkerTest ARRAY_INLINE_ROWS, RowSignature.builder() .addTimeColumn() - .add("s", ValueType.STRING) - .add("n", ValueType.LONG) - .add("ad", ValueType.DOUBLE_ARRAY) - .add("al", ValueType.LONG_ARRAY) - .add("as", ValueType.STRING_ARRAY) + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) + .add("ad", ColumnType.DOUBLE_ARRAY) + .add("al", ColumnType.LONG_ARRAY) + .add("as", ColumnType.STRING_ARRAY) .build() ); @@ -195,8 +195,8 @@ public class ClientQuerySegmentWalkerTest ARRAY_INLINE_ROWS, RowSignature.builder() .addTimeColumn() - .add("s", ValueType.STRING) - .add("n", ValueType.LONG) + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) .add("ad", null) .add("al", null) .add("as", null) @@ -351,7 +351,7 @@ public void testTimeseriesOnGroupByOnTable() query.withDataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{"x"}, new Object[]{"y"}, new Object[]{"z"}), - RowSignature.builder().add("s", ValueType.STRING).build() + RowSignature.builder().add("s", ColumnType.STRING).build() ) ) ) @@ -521,7 +521,7 @@ public void testJoinOnGroupByOnTable() query.getDataSource().getChildren().get(0), InlineDataSource.fromIterable( ImmutableList.of(new Object[]{"y"}), - RowSignature.builder().add("s", ValueType.STRING).build() + RowSignature.builder().add("s", ColumnType.STRING).build() ) ) ) @@ -598,7 +598,7 @@ public void testJoinOnGroupByOnUnionOfTables() unionDataSource.getChildren().get(0), InlineDataSource.fromIterable( ImmutableList.of(new Object[]{"y"}), - RowSignature.builder().add("s", ValueType.STRING).build() + RowSignature.builder().add("s", ColumnType.STRING).build() ) ) ) @@ -611,7 +611,7 @@ public void testJoinOnGroupByOnUnionOfTables() unionDataSource.getChildren().get(1), InlineDataSource.fromIterable( ImmutableList.of(new Object[]{"y"}), - RowSignature.builder().add("s", ValueType.STRING).build() + RowSignature.builder().add("s", ColumnType.STRING).build() ) ) ) @@ -812,7 +812,7 @@ public void testGroupByOnArraysDoubles() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [DOUBLE_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -834,7 +834,7 @@ public void testGroupByOnArraysUnknownDoubles() .withId("queryId"); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, @@ -869,7 +869,7 @@ public void testGroupByOnArraysLongs() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [LONG_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -891,7 +891,7 @@ public void testGroupByOnArraysUnknownLongs() .withId("queryId"); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, @@ -927,7 +927,7 @@ public void testGroupByOnArraysStrings() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [STRING_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -949,7 +949,7 @@ public void testGroupByOnArraysUnknownStrings() .withId("queryId"); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, @@ -988,7 +988,7 @@ public void testTopNArraysDoubles() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [DOUBLE_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -1012,7 +1012,7 @@ public void testTopNOnArraysUnknownDoubles() .withId(UUID.randomUUID().toString()); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, @@ -1049,7 +1049,7 @@ public void testTopNOnArraysLongs() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [LONG_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -1073,7 +1073,7 @@ public void testTopNOnArraysUnknownLongs() .withId(UUID.randomUUID().toString()); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, @@ -1111,7 +1111,7 @@ public void testTopNOnArraysStrings() // group by cannot handle true array types, expect this, RuntimeExeception with IAE in stack trace expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [STRING_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( query, @@ -1135,7 +1135,7 @@ public void testTopNOnArraysUnknownStrings() .withId(UUID.randomUUID().toString()); - // 'unknown' is treated as ValueType.STRING. this might not always be the case, so this is a test case of wacky + // 'unknown' is treated as ColumnType.STRING. this might not always be the case, so this is a test case of wacky // behavior of sorts testQuery( query, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregations.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregations.java index 67215833c77b..44331a66bfe4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregations.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregations.java @@ -104,7 +104,7 @@ public static DruidExpression toDruidExpressionForNumericAggregator( if (druidExpression.isSimpleExtraction() && (!druidExpression.isDirectColumnAccess() - || rowSignature.getColumnType(druidExpression.getDirectColumn()).orElse(null) == ValueType.STRING)) { + || rowSignature.getColumnType(druidExpression.getDirectColumn()).map(type -> type.is(ValueType.STRING)).orElse(false))) { // Aggregators are unable to implicitly cast strings to numbers. // So remove the simple extraction, which forces the expression to be used instead of the direct column access. return druidExpression.map(simpleExtraction -> null, Function.identity()); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/DimensionExpression.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/DimensionExpression.java index 10b5bbefe345..c352b4705518 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/DimensionExpression.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/DimensionExpression.java @@ -22,7 +22,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.expression.DruidExpression; import java.util.Objects; @@ -35,7 +35,7 @@ public class DimensionExpression public static DimensionExpression ofSimpleColumn( final String outputName, final DruidExpression expression, - final ValueType outputType + final ColumnType outputType ) { return new DimensionExpression(outputName, outputName, expression, outputType); @@ -48,7 +48,7 @@ public static DimensionExpression ofVirtualColumn( final String virtualColumn, final String outputName, final DruidExpression expression, - final ValueType outputType + final ColumnType outputType ) { return new DimensionExpression(virtualColumn, outputName, expression, outputType); @@ -57,13 +57,13 @@ public static DimensionExpression ofVirtualColumn( private final String virtualColumn; private final String outputName; private final DruidExpression expression; - private final ValueType outputType; + private final ColumnType outputType; private DimensionExpression( final String virtualColumn, final String outputName, final DruidExpression expression, - final ValueType outputType + final ColumnType outputType ) { Preconditions.checkArgument(!expression.isSimpleExtraction() || outputName.equals(virtualColumn)); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ApproxCountDistinctSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ApproxCountDistinctSqlAggregator.java index bbf01891fc98..761c8d9f267b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ApproxCountDistinctSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ApproxCountDistinctSqlAggregator.java @@ -41,6 +41,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; @@ -97,11 +98,11 @@ public Aggregation toDruidAggregation( final String aggregatorName = finalizeAggregations ? Calcites.makePrefixedName(name, "a") : name; if (arg.isDirectColumnAccess() - && rowSignature.getColumnType(arg.getDirectColumn()).orElse(null) == ValueType.COMPLEX) { + && rowSignature.getColumnType(arg.getDirectColumn()).map(type -> type.is(ValueType.COMPLEX)).orElse(false)) { aggregatorFactory = new HyperUniquesAggregatorFactory(aggregatorName, arg.getDirectColumn(), false, true); } else { final RelDataType dataType = rexNode.getType(); - final ValueType inputType = Calcites.getValueTypeForRelDataType(dataType); + final ColumnType inputType = Calcites.getColumnTypeForRelDataType(dataType); if (inputType == null) { throw new ISE( "Cannot translate sqlTypeName[%s] to Druid type for field[%s]", diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java index 9359b6ea9447..d15860be981e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java @@ -42,8 +42,8 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -112,24 +112,24 @@ public Aggregation toDruidAggregation( final String fieldName; final String initialvalue; - final ValueType elementType; - final ValueType druidType = Calcites.getValueTypeForRelDataTypeFull(aggregateCall.getType()); - if (druidType == null) { + final ColumnType druidType = Calcites.getValueTypeForRelDataTypeFull(aggregateCall.getType()); + final ColumnType elementType; + if (druidType == null || !druidType.isArray()) { initialvalue = "[]"; - elementType = ValueType.STRING; + elementType = ColumnType.STRING; } else { - switch (druidType) { - case LONG_ARRAY: + elementType = (ColumnType) druidType.getElementType(); + // elementType should never be null if druidType.isArray is true + assert elementType != null; + switch (elementType.getType()) { + case LONG: initialvalue = "[]"; - elementType = ValueType.LONG; break; - case DOUBLE_ARRAY: + case DOUBLE: initialvalue = "[]"; - elementType = ValueType.DOUBLE; break; default: initialvalue = "[]"; - elementType = ValueType.STRING; break; } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java index a6fe93cb2def..cd55c9290307 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java @@ -39,8 +39,8 @@ import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -149,7 +149,7 @@ public Aggregation toDruidAggregation( if (arg.isDirectColumnAccess()) { fieldName = arg.getDirectColumn(); } else { - VirtualColumn vc = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, arg, ValueType.LONG); + VirtualColumn vc = virtualColumnRegistry.getOrCreateVirtualColumnForExpression(plannerContext, arg, ColumnType.LONG); fieldName = vc.getOutputName(); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java index be9a69f26839..7bf9946080ba 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/EarliestLatestAnySqlAggregator.java @@ -51,8 +51,8 @@ import org.apache.druid.query.aggregation.last.StringLastAggregatorFactory; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -76,9 +76,9 @@ enum AggregatorType { EARLIEST { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory(String name, String fieldName, ColumnType type, int maxStringBytes) { - switch (type) { + switch (type.getType()) { case LONG: return new LongFirstAggregatorFactory(name, fieldName); case FLOAT: @@ -96,9 +96,9 @@ AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueTy LATEST { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory(String name, String fieldName, ColumnType type, int maxStringBytes) { - switch (type) { + switch (type.getType()) { case LONG: return new LongLastAggregatorFactory(name, fieldName); case FLOAT: @@ -116,9 +116,9 @@ AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueTy ANY_VALUE { @Override - AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueType type, int maxStringBytes) + AggregatorFactory createAggregatorFactory(String name, String fieldName, ColumnType type, int maxStringBytes) { - switch (type) { + switch (type.getType()) { case LONG: return new LongAnyAggregatorFactory(name, fieldName); case FLOAT: @@ -136,7 +136,7 @@ AggregatorFactory createAggregatorFactory(String name, String fieldName, ValueTy abstract AggregatorFactory createAggregatorFactory( String name, String fieldName, - ValueType outputType, + ColumnType outputType, int maxStringBytes ); } @@ -197,7 +197,7 @@ public Aggregation toDruidAggregation( // Second arg must be a literal, if it exists (the type signature below requires it). final int maxBytes = rexNodes.size() > 1 ? RexLiteral.intValue(rexNodes.get(1)) : -1; - final ValueType outputType = Calcites.getValueTypeForRelDataType(aggregateCall.getType()); + final ColumnType outputType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); if (outputType == null) { throw new ISE( "Cannot translate output sqlTypeName[%s] to Druid type for aggregator[%s]", diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java index 302560c9bf47..06633d8cf6a5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MaxSqlAggregator.java @@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.FloatMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; @@ -49,8 +50,11 @@ Aggregation getAggregation( final String expression ) { - final ValueType valueType = Calcites.getValueTypeForRelDataType(aggregateCall.getType()); - return Aggregation.create(createMaxAggregatorFactory(valueType, name, fieldName, expression, macroTable)); + final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); + if (valueType == null) { + return null; + } + return Aggregation.create(createMaxAggregatorFactory(valueType.getType(), name, fieldName, expression, macroTable)); } private static AggregatorFactory createMaxAggregatorFactory( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java index 9e41ce5474b9..373a7c683ac0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MinSqlAggregator.java @@ -28,7 +28,7 @@ import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.query.aggregation.FloatMinAggregatorFactory; import org.apache.druid.query.aggregation.LongMinAggregatorFactory; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; @@ -49,19 +49,19 @@ Aggregation getAggregation( final String expression ) { - final ValueType valueType = Calcites.getValueTypeForRelDataType(aggregateCall.getType()); + final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); return Aggregation.create(createMinAggregatorFactory(valueType, name, fieldName, expression, macroTable)); } private static AggregatorFactory createMinAggregatorFactory( - final ValueType aggregationType, + final ColumnType aggregationType, final String name, final String fieldName, final String expression, final ExprMacroTable macroTable ) { - switch (aggregationType) { + switch (aggregationType.getType()) { case LONG: return new LongMinAggregatorFactory(name, fieldName, expression, macroTable); case FLOAT: diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java index 78a3e160f1ce..a3383564ba23 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/MultiColumnSqlAggregator.java @@ -27,8 +27,8 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.Aggregations; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; @@ -122,7 +122,7 @@ private Aggregation getAggregation( List fieldInfoList ) { - final ValueType valueType = Calcites.getValueTypeForRelDataType(aggregateCall.getType()); + final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); List aggregatorFactories = new ArrayList<>(); List postAggregators = new ArrayList<>(); @@ -140,13 +140,13 @@ private Aggregation getAggregation( } abstract AggregatorFactory createAggregatorFactory( - ValueType valueType, + ColumnType valueType, String prefixedName, FieldInfo fieldInfo, ExprMacroTable macroTable); abstract PostAggregator createFinalPostAggregator( - ValueType valueType, + ColumnType valueType, String name, List postAggregators); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index 596f69b13299..c11f5a0602e8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -41,8 +41,8 @@ import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -125,7 +125,7 @@ public Aggregation toDruidAggregation( final ExprMacroTable macroTable = plannerContext.getExprMacroTable(); final String initialvalue = "[]"; - final ValueType elementType = ValueType.STRING; + final ColumnType elementType = ColumnType.STRING; final String fieldName; if (arg.isDirectColumnAccess()) { fieldName = arg.getDirectColumn(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java index b9a5af57d003..bea89124544d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/SumSqlAggregator.java @@ -28,6 +28,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.planner.Calcites; @@ -49,8 +50,11 @@ Aggregation getAggregation( final String expression ) { - final ValueType valueType = Calcites.getValueTypeForRelDataType(aggregateCall.getType()); - return Aggregation.create(createSumAggregatorFactory(valueType, name, fieldName, expression, macroTable)); + final ColumnType valueType = Calcites.getColumnTypeForRelDataType(aggregateCall.getType()); + if (valueType == null) { + return null; + } + return Aggregation.create(createSumAggregatorFactory(valueType.getType(), name, fieldName, expression, macroTable)); } static AggregatorFactory createSumAggregatorFactory( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java index fc51ecdb5a1a..be38c7b2fa26 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/DruidExpression.java @@ -28,7 +28,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import javax.annotation.Nullable; @@ -55,9 +55,9 @@ public class DruidExpression @Nullable private final SimpleExtraction simpleExtraction; private final String expression; - private final TrinaryFn virtualColumnFn; + private final TrinaryFn virtualColumnFn; - private DruidExpression(@Nullable final SimpleExtraction simpleExtraction, final String expression, @Nullable final TrinaryFn virtualColumnFn) + private DruidExpression(@Nullable final SimpleExtraction simpleExtraction, final String expression, @Nullable final TrinaryFn virtualColumnFn) { this.simpleExtraction = simpleExtraction; this.expression = Preconditions.checkNotNull(expression); @@ -89,7 +89,7 @@ public static DruidExpression fromFunctionCall(final String functionName, final public static DruidExpression forVirtualColumn( final String expression, - final TrinaryFn virtualColumnFunction + final TrinaryFn virtualColumnFunction ) { return new DruidExpression(null, expression, virtualColumnFunction); @@ -182,7 +182,7 @@ public SimpleExtraction getSimpleExtraction() public VirtualColumn toVirtualColumn( final String name, - final ValueType outputType, + final ColumnType outputType, final ExprMacroTable macroTable ) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/SimpleExtraction.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/SimpleExtraction.java index 867920267f6c..81d19d1b767c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/SimpleExtraction.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/SimpleExtraction.java @@ -25,7 +25,7 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; import java.util.Objects; @@ -73,7 +73,7 @@ public SimpleExtraction cascade(final ExtractionFn nextExtractionFn) public DimensionSpec toDimensionSpec( final String outputName, - final ValueType outputType + final ColumnType outputType ) { Preconditions.checkNotNull(outputType, "outputType"); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReductionOperatorConversionHelper.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReductionOperatorConversionHelper.java index edfaa6bb3787..4a457b7ad7e4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReductionOperatorConversionHelper.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReductionOperatorConversionHelper.java @@ -24,7 +24,8 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.math.expr.ExprTypeConversion; +import org.apache.druid.math.expr.ExpressionTypeConversion; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.planner.Calcites; @@ -39,7 +40,7 @@ private ReductionOperatorConversionHelper() * https://dev.mysql.com/doc/refman/8.0/en/comparison-operators.html#function_least * * @see org.apache.druid.math.expr.Function.ReduceFunction#apply - * @see ExprTypeConversion#function + * @see ExpressionTypeConversion#function */ static final SqlReturnTypeInference TYPE_INFERENCE = opBinding -> { @@ -56,17 +57,19 @@ private ReductionOperatorConversionHelper() for (int i = 0; i < n; i++) { RelDataType type = opBinding.getOperandType(i); SqlTypeName sqlTypeName = type.getSqlTypeName(); - ValueType valueType = Calcites.getValueTypeForRelDataType(type); + ColumnType valueType = Calcites.getColumnTypeForRelDataType(type); // Return types are listed in order of preference: - if (valueType == ValueType.STRING) { - returnSqlTypeName = sqlTypeName; - break; - } else if (valueType == ValueType.DOUBLE || valueType == ValueType.FLOAT) { - returnSqlTypeName = SqlTypeName.DOUBLE; - hasDouble = true; - } else if (valueType == ValueType.LONG && !hasDouble) { - returnSqlTypeName = SqlTypeName.BIGINT; + if (valueType != null) { + if (valueType.is(ValueType.STRING)) { + returnSqlTypeName = sqlTypeName; + break; + } else if (valueType.anyOf(ValueType.DOUBLE, ValueType.FLOAT)) { + returnSqlTypeName = SqlTypeName.DOUBLE; + hasDouble = true; + } else if (valueType.is(ValueType.LONG) && !hasDouble) { + returnSqlTypeName = SqlTypeName.BIGINT; + } } else if (sqlTypeName != SqlTypeName.NULL) { throw new IAE("Argument %d has invalid type: %s", i, sqlTypeName); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java index 87b731713031..bcfac162683c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java @@ -40,7 +40,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.calcite.table.RowSignatures; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; @@ -133,11 +135,11 @@ public static String escapeStringLiteral(final String s) * which returns the most accurate conversion of {@link RelDataType} to {@link ValueType}. */ @Nullable - public static ValueType getValueTypeForRelDataType(final RelDataType type) + public static ColumnType getColumnTypeForRelDataType(final RelDataType type) { - ValueType valueType = getValueTypeForRelDataTypeFull(type); - if (ValueType.isArray(valueType)) { - return ValueType.STRING; + ColumnType valueType = getValueTypeForRelDataTypeFull(type); + if (valueType != null && valueType.isArray()) { + return ColumnType.STRING; } return valueType; } @@ -146,28 +148,31 @@ public static ValueType getValueTypeForRelDataType(final RelDataType type) * Convert {@link RelDataType} to the most appropriate {@link ValueType} */ @Nullable - public static ValueType getValueTypeForRelDataTypeFull(final RelDataType type) + public static ColumnType getValueTypeForRelDataTypeFull(final RelDataType type) { final SqlTypeName sqlTypeName = type.getSqlTypeName(); if (SqlTypeName.FLOAT == sqlTypeName) { - return ValueType.FLOAT; + return ColumnType.FLOAT; } else if (isDoubleType(sqlTypeName)) { - return ValueType.DOUBLE; + return ColumnType.DOUBLE; } else if (isLongType(sqlTypeName)) { - return ValueType.LONG; + return ColumnType.LONG; } else if (SqlTypeName.CHAR_TYPES.contains(sqlTypeName)) { - return ValueType.STRING; + return ColumnType.STRING; } else if (SqlTypeName.OTHER == sqlTypeName) { - return ValueType.COMPLEX; + if (type instanceof RowSignatures.ComplexSqlType) { + return ColumnType.ofComplex(((RowSignatures.ComplexSqlType) type).getComplexTypeName()); + } + return ColumnType.UNKNOWN_COMPLEX; } else if (sqlTypeName == SqlTypeName.ARRAY) { SqlTypeName componentType = type.getComponentType().getSqlTypeName(); if (isDoubleType(componentType)) { - return ValueType.DOUBLE_ARRAY; + return ColumnType.DOUBLE_ARRAY; } if (isLongType(componentType)) { - return ValueType.LONG_ARRAY; + return ColumnType.LONG_ARRAY; } - return ValueType.STRING_ARRAY; + return ColumnType.STRING_ARRAY; } else { return null; } @@ -187,15 +192,15 @@ public static boolean isLongType(SqlTypeName sqlTypeName) public static StringComparator getStringComparatorForRelDataType(RelDataType dataType) { - final ValueType valueType = getValueTypeForRelDataType(dataType); + final ColumnType valueType = getColumnTypeForRelDataType(dataType); return getStringComparatorForValueType(valueType); } - public static StringComparator getStringComparatorForValueType(ValueType valueType) + public static StringComparator getStringComparatorForValueType(ColumnType valueType) { - if (ValueType.isNumeric(valueType)) { + if (valueType.isNumeric()) { return StringComparators.NUMERIC; - } else if (ValueType.STRING == valueType) { + } else if (valueType.is(ValueType.STRING)) { return StringComparators.LEXICOGRAPHIC; } else { throw new ISE("Unrecognized valueType[%s]", valueType); @@ -238,7 +243,6 @@ public static RelDataType createSqlTypeWithNullability( default: dataType = typeFactory.createSqlType(typeName); } - return typeFactory.createTypeWithNullability(dataType, nullable); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java index 0f1988a9b241..a27871d4c779 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java @@ -119,7 +119,7 @@ public void reduce( if (exprResult.isNumericNull()) { literal = rexBuilder.makeNullLiteral(constExp.getType()); } else { - if (exprResult.type() == ExprType.LONG) { + if (exprResult.type().is(ExprType.LONG)) { bigDecimal = BigDecimal.valueOf(exprResult.asLong()); } else { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index c1ca4ae2a3dc..cb674d2012ee 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -69,7 +69,9 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.aggregation.Aggregation; import org.apache.druid.sql.calcite.aggregation.DimensionExpression; @@ -395,8 +397,8 @@ private static List computeDimensions( } final RelDataType dataType = rexNode.getType(); - final ValueType outputType = Calcites.getValueTypeForRelDataType(dataType); - if (outputType == null || outputType == ValueType.COMPLEX) { + final ColumnType outputType = Calcites.getColumnTypeForRelDataType(dataType); + if (Types.isNullOr(outputType, ValueType.COMPLEX)) { // Can't group on unknown or COMPLEX types. throw new CannotBuildQueryException(aggregate, rexNode); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java index 631b338f6816..89c673cb8c9e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Projection.java @@ -27,11 +27,12 @@ import org.apache.calcite.sql.SqlKind; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.expression.DruidExpression; @@ -261,9 +262,12 @@ public static Projection preAggregation( final DruidExpression expression = expressions.get(i); final RelDataType dataType = project.getRowType().getFieldList().get(i).getType(); - if (expression.isDirectColumnAccess() - && inputRowSignature.getColumnType(expression.getDirectColumn()).orElse(null) - == Calcites.getValueTypeForRelDataType(dataType)) { + if (expression.isDirectColumnAccess() && + Objects.equals( + inputRowSignature.getColumnType(expression.getDirectColumn()).orElse(null), + Calcites.getColumnTypeForRelDataType(dataType) + ) + ) { // Refer to column directly when it's a direct access with matching type. rowOrder.add(expression.getDirectColumn()); } else { @@ -305,20 +309,20 @@ private static boolean postAggregatorDirectColumnIsOk( } // We don't really have a way to cast complex type. So might as well not do anything and return. - final ValueType columnValueType = + final ColumnType columnValueType = aggregateRowSignature.getColumnType(expression.getDirectColumn()) .orElseThrow( () -> new ISE("Encountered null type for column[%s]", expression.getDirectColumn()) ); - if (columnValueType == ValueType.COMPLEX) { + if (columnValueType.is(ValueType.COMPLEX)) { return true; } // Check if a cast is necessary. - final ExprType toExprType = ExprType.fromValueTypeStrict(columnValueType); - final ExprType fromExprType = ExprType.fromValueTypeStrict( - Calcites.getValueTypeForRelDataType(rexNode.getType()) + final ExpressionType toExprType = ExpressionType.fromColumnTypeStrict(columnValueType); + final ExpressionType fromExprType = ExpressionType.fromColumnTypeStrict( + Calcites.getColumnTypeForRelDataType(rexNode.getType()) ); return toExprType.equals(fromExprType); @@ -345,15 +349,15 @@ private static boolean postAggregatorComplexDirectColumnIsOk( } // Check if a cast is necessary. - final ValueType toValueType = + final ColumnType toValueType = aggregateRowSignature.getColumnType(expression.getDirectColumn()) .orElseThrow( () -> new ISE("Encountered null type for column[%s]", expression.getDirectColumn()) ); - final ValueType fromValueType = Calcites.getValueTypeForRelDataType(rexNode.getType()); + final ColumnType fromValueType = Calcites.getColumnTypeForRelDataType(rexNode.getType()); - return toValueType == ValueType.COMPLEX && fromValueType == ValueType.COMPLEX; + return toValueType.is(ValueType.COMPLEX) && toValueType.equals(fromValueType); } public List getPostAggregators() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java index bdfa396b9c83..466ebff753b7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java @@ -21,6 +21,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.segment.VirtualColumn; +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.sql.calcite.expression.DruidExpression; @@ -83,7 +84,7 @@ public boolean isVirtualColumnDefined(String virtualColumnName) public VirtualColumn getOrCreateVirtualColumnForExpression( PlannerContext plannerContext, DruidExpression expression, - ValueType valueType + ColumnType valueType ) { ExpressionWrapper expressionWrapper = new ExpressionWrapper(expression.getExpression(), valueType); @@ -119,7 +120,7 @@ public VirtualColumn getOrCreateVirtualColumnForExpression( return getOrCreateVirtualColumnForExpression( plannerContext, expression, - Calcites.getValueTypeForRelDataType(dataType) + Calcites.getColumnTypeForRelDataType(dataType) ); } @@ -135,7 +136,7 @@ public VirtualColumn getVirtualColumn(String virtualColumnName) @Nullable public VirtualColumn getVirtualColumnByExpression(String expression, RelDataType type) { - ExpressionWrapper expressionWrapper = new ExpressionWrapper(expression, Calcites.getValueTypeForRelDataType(type)); + ExpressionWrapper expressionWrapper = new ExpressionWrapper(expression, Calcites.getColumnTypeForRelDataType(type)); return virtualColumnsByExpression.get(expressionWrapper); } @@ -151,7 +152,7 @@ public RowSignature getFullRowSignature() for (VirtualColumn virtualColumn : virtualColumnsByName.values()) { final String columnName = virtualColumn.getOutputName(); - builder.add(columnName, virtualColumn.capabilities(baseSignature, columnName).getType()); + builder.add(columnName, virtualColumn.capabilities(baseSignature, columnName).toColumnType()); } return builder.build(); @@ -171,9 +172,9 @@ public List findVirtualColumns(List allColumns) private static class ExpressionWrapper { private final String expression; - private final ValueType valueType; + private final ColumnType valueType; - public ExpressionWrapper(String expression, ValueType valueType) + public ExpressionWrapper(String expression, ColumnType valueType) { this.expression = expression; this.valueType = valueType; @@ -189,7 +190,7 @@ public boolean equals(Object o) return false; } ExpressionWrapper expressionWrapper = (ExpressionWrapper) o; - return Objects.equals(expression, expressionWrapper.expression) && valueType == expressionWrapper.valueType; + return Objects.equals(expression, expressionWrapper.expression) && Objects.equals(valueType, expressionWrapper.valueType); } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 70e154fa2e40..0d01d6fa678e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -38,7 +38,6 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -53,8 +52,8 @@ import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +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.join.JoinableFactory; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.SegmentManager; @@ -756,7 +755,7 @@ private Set refreshSegmentsForDataSource(final String dataSource, fin DruidTable buildDruidTable(final String dataSource) { ConcurrentSkipListMap segmentsMap = segmentMetadataInfo.get(dataSource); - final Map columnTypes = new TreeMap<>(); + final Map columnTypes = new TreeMap<>(); if (segmentsMap != null) { for (AvailableSegmentMetadata availableSegmentMetadata : segmentsMap.values()) { @@ -764,7 +763,7 @@ DruidTable buildDruidTable(final String dataSource) if (rowSignature != null) { for (String column : rowSignature.getColumnNames()) { // Newer column types should override older ones. - final ValueType columnType = + final ColumnType columnType = rowSignature.getColumnType(column) .orElseThrow(() -> new ISE("Encountered null type for column[%s]", column)); @@ -885,14 +884,17 @@ private static RowSignature analysisToRowSignature(final SegmentAnalysis analysi continue; } - ValueType valueType; + ColumnType valueType = null; try { - valueType = ValueType.valueOf(StringUtils.toUpperCase(entry.getValue().getType())); + valueType = ColumnType.fromString(entry.getValue().getType()); } - catch (IllegalArgumentException e) { - // Assume unrecognized types are some flavor of COMPLEX. This throws away information about exactly - // what kind of complex column it is, which we may want to preserve some day. - valueType = ValueType.COMPLEX; + catch (IllegalArgumentException ignored) { + } + + // Assume unrecognized types are some flavor of COMPLEX. This throws away information about exactly + // what kind of complex column it is, which we may want to preserve some day. + if (valueType == null) { + valueType = ColumnType.UNKNOWN_COMPLEX; } rowSignatureBuilder.add(entry.getKey(), valueType); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java index 96fe42c07160..e0f86f0afde5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java @@ -46,8 +46,8 @@ import org.apache.calcite.schema.impl.AbstractTable; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.java.util.emitter.EmittingLogger; +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.server.security.Action; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.AuthorizationUtils; @@ -74,42 +74,42 @@ public class InformationSchema extends AbstractSchema private static final String COLUMNS_TABLE = "COLUMNS"; private static final RowSignature SCHEMATA_SIGNATURE = RowSignature .builder() - .add("CATALOG_NAME", ValueType.STRING) - .add("SCHEMA_NAME", ValueType.STRING) - .add("SCHEMA_OWNER", ValueType.STRING) - .add("DEFAULT_CHARACTER_SET_CATALOG", ValueType.STRING) - .add("DEFAULT_CHARACTER_SET_SCHEMA", ValueType.STRING) - .add("DEFAULT_CHARACTER_SET_NAME", ValueType.STRING) - .add("SQL_PATH", ValueType.STRING) + .add("CATALOG_NAME", ColumnType.STRING) + .add("SCHEMA_NAME", ColumnType.STRING) + .add("SCHEMA_OWNER", ColumnType.STRING) + .add("DEFAULT_CHARACTER_SET_CATALOG", ColumnType.STRING) + .add("DEFAULT_CHARACTER_SET_SCHEMA", ColumnType.STRING) + .add("DEFAULT_CHARACTER_SET_NAME", ColumnType.STRING) + .add("SQL_PATH", ColumnType.STRING) .build(); private static final RowSignature TABLES_SIGNATURE = RowSignature .builder() - .add("TABLE_CATALOG", ValueType.STRING) - .add("TABLE_SCHEMA", ValueType.STRING) - .add("TABLE_NAME", ValueType.STRING) - .add("TABLE_TYPE", ValueType.STRING) - .add("IS_JOINABLE", ValueType.STRING) - .add("IS_BROADCAST", ValueType.STRING) + .add("TABLE_CATALOG", ColumnType.STRING) + .add("TABLE_SCHEMA", ColumnType.STRING) + .add("TABLE_NAME", ColumnType.STRING) + .add("TABLE_TYPE", ColumnType.STRING) + .add("IS_JOINABLE", ColumnType.STRING) + .add("IS_BROADCAST", ColumnType.STRING) .build(); private static final RowSignature COLUMNS_SIGNATURE = RowSignature .builder() - .add("TABLE_CATALOG", ValueType.STRING) - .add("TABLE_SCHEMA", ValueType.STRING) - .add("TABLE_NAME", ValueType.STRING) - .add("COLUMN_NAME", ValueType.STRING) - .add("ORDINAL_POSITION", ValueType.STRING) - .add("COLUMN_DEFAULT", ValueType.STRING) - .add("IS_NULLABLE", ValueType.STRING) - .add("DATA_TYPE", ValueType.STRING) - .add("CHARACTER_MAXIMUM_LENGTH", ValueType.STRING) - .add("CHARACTER_OCTET_LENGTH", ValueType.STRING) - .add("NUMERIC_PRECISION", ValueType.STRING) - .add("NUMERIC_PRECISION_RADIX", ValueType.STRING) - .add("NUMERIC_SCALE", ValueType.STRING) - .add("DATETIME_PRECISION", ValueType.STRING) - .add("CHARACTER_SET_NAME", ValueType.STRING) - .add("COLLATION_NAME", ValueType.STRING) - .add("JDBC_TYPE", ValueType.LONG) + .add("TABLE_CATALOG", ColumnType.STRING) + .add("TABLE_SCHEMA", ColumnType.STRING) + .add("TABLE_NAME", ColumnType.STRING) + .add("COLUMN_NAME", ColumnType.STRING) + .add("ORDINAL_POSITION", ColumnType.STRING) + .add("COLUMN_DEFAULT", ColumnType.STRING) + .add("IS_NULLABLE", ColumnType.STRING) + .add("DATA_TYPE", ColumnType.STRING) + .add("CHARACTER_MAXIMUM_LENGTH", ColumnType.STRING) + .add("CHARACTER_OCTET_LENGTH", ColumnType.STRING) + .add("NUMERIC_PRECISION", ColumnType.STRING) + .add("NUMERIC_PRECISION_RADIX", ColumnType.STRING) + .add("NUMERIC_SCALE", ColumnType.STRING) + .add("DATETIME_PRECISION", ColumnType.STRING) + .add("CHARACTER_SET_NAME", ColumnType.STRING) + .add("COLLATION_NAME", ColumnType.STRING) + .add("JDBC_TYPE", ColumnType.LONG) .build(); private static final RelDataTypeSystem TYPE_SYSTEM = RelDataTypeSystem.DEFAULT; @@ -420,6 +420,8 @@ public Object[] apply(final RelDataTypeField field) boolean isNumeric = SqlTypeName.NUMERIC_TYPES.contains(type.getSqlTypeName()); boolean isCharacter = SqlTypeName.CHAR_TYPES.contains(type.getSqlTypeName()); boolean isDateTime = SqlTypeName.DATETIME_TYPES.contains(type.getSqlTypeName()); + + final String typeName = type instanceof RowSignatures.ComplexSqlType ? ((RowSignatures.ComplexSqlType) type).asTypeString() : type.getSqlTypeName().toString(); return new Object[]{ CATALOG_NAME, // TABLE_CATALOG schemaName, // TABLE_SCHEMA @@ -428,7 +430,7 @@ public Object[] apply(final RelDataTypeField field) String.valueOf(field.getIndex()), // ORDINAL_POSITION "", // COLUMN_DEFAULT type.isNullable() ? INFO_TRUE : INFO_FALSE, // IS_NULLABLE - type.getSqlTypeName().toString(), // DATA_TYPE + typeName, // DATA_TYPE null, // CHARACTER_MAXIMUM_LENGTH null, // CHARACTER_OCTET_LENGTH isNumeric ? String.valueOf(type.getPrecision()) : null, // NUMERIC_PRECISION diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java index 6ddeaabb445d..b3400e7afca8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java @@ -25,8 +25,8 @@ import org.apache.calcite.schema.impl.AbstractSchema; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +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.join.lookup.LookupColumnSelectorFactory; import org.apache.druid.sql.calcite.table.DruidTable; @@ -39,8 +39,8 @@ public class LookupSchema extends AbstractSchema { private static final RowSignature ROW_SIGNATURE = RowSignature.builder() - .add(LookupColumnSelectorFactory.KEY_COLUMN, ValueType.STRING) - .add(LookupColumnSelectorFactory.VALUE_COLUMN, ValueType.STRING) + .add(LookupColumnSelectorFactory.KEY_COLUMN, ColumnType.STRING) + .add(LookupColumnSelectorFactory.VALUE_COLUMN, ColumnType.STRING) .build(); private final LookupExtractorFactoryContainerProvider lookupProvider; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 0562e3333142..42b707ef2724 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -67,8 +67,8 @@ import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHandler; import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHolder; +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.server.DruidNode; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; @@ -131,72 +131,72 @@ public class SystemSchema extends AbstractSchema static final RowSignature SEGMENTS_SIGNATURE = RowSignature .builder() - .add("segment_id", ValueType.STRING) - .add("datasource", ValueType.STRING) - .add("start", ValueType.STRING) - .add("end", ValueType.STRING) - .add("size", ValueType.LONG) - .add("version", ValueType.STRING) - .add("partition_num", ValueType.LONG) - .add("num_replicas", ValueType.LONG) - .add("num_rows", ValueType.LONG) - .add("is_published", ValueType.LONG) - .add("is_available", ValueType.LONG) - .add("is_realtime", ValueType.LONG) - .add("is_overshadowed", ValueType.LONG) - .add("shard_spec", ValueType.STRING) - .add("dimensions", ValueType.STRING) - .add("metrics", ValueType.STRING) - .add("last_compaction_state", ValueType.STRING) + .add("segment_id", ColumnType.STRING) + .add("datasource", ColumnType.STRING) + .add("start", ColumnType.STRING) + .add("end", ColumnType.STRING) + .add("size", ColumnType.LONG) + .add("version", ColumnType.STRING) + .add("partition_num", ColumnType.LONG) + .add("num_replicas", ColumnType.LONG) + .add("num_rows", ColumnType.LONG) + .add("is_published", ColumnType.LONG) + .add("is_available", ColumnType.LONG) + .add("is_realtime", ColumnType.LONG) + .add("is_overshadowed", ColumnType.LONG) + .add("shard_spec", ColumnType.STRING) + .add("dimensions", ColumnType.STRING) + .add("metrics", ColumnType.STRING) + .add("last_compaction_state", ColumnType.STRING) .build(); static final RowSignature SERVERS_SIGNATURE = RowSignature .builder() - .add("server", ValueType.STRING) - .add("host", ValueType.STRING) - .add("plaintext_port", ValueType.LONG) - .add("tls_port", ValueType.LONG) - .add("server_type", ValueType.STRING) - .add("tier", ValueType.STRING) - .add("curr_size", ValueType.LONG) - .add("max_size", ValueType.LONG) - .add("is_leader", ValueType.LONG) + .add("server", ColumnType.STRING) + .add("host", ColumnType.STRING) + .add("plaintext_port", ColumnType.LONG) + .add("tls_port", ColumnType.LONG) + .add("server_type", ColumnType.STRING) + .add("tier", ColumnType.STRING) + .add("curr_size", ColumnType.LONG) + .add("max_size", ColumnType.LONG) + .add("is_leader", ColumnType.LONG) .build(); static final RowSignature SERVER_SEGMENTS_SIGNATURE = RowSignature .builder() - .add("server", ValueType.STRING) - .add("segment_id", ValueType.STRING) + .add("server", ColumnType.STRING) + .add("segment_id", ColumnType.STRING) .build(); static final RowSignature TASKS_SIGNATURE = RowSignature .builder() - .add("task_id", ValueType.STRING) - .add("group_id", ValueType.STRING) - .add("type", ValueType.STRING) - .add("datasource", ValueType.STRING) - .add("created_time", ValueType.STRING) - .add("queue_insertion_time", ValueType.STRING) - .add("status", ValueType.STRING) - .add("runner_status", ValueType.STRING) - .add("duration", ValueType.LONG) - .add("location", ValueType.STRING) - .add("host", ValueType.STRING) - .add("plaintext_port", ValueType.LONG) - .add("tls_port", ValueType.LONG) - .add("error_msg", ValueType.STRING) + .add("task_id", ColumnType.STRING) + .add("group_id", ColumnType.STRING) + .add("type", ColumnType.STRING) + .add("datasource", ColumnType.STRING) + .add("created_time", ColumnType.STRING) + .add("queue_insertion_time", ColumnType.STRING) + .add("status", ColumnType.STRING) + .add("runner_status", ColumnType.STRING) + .add("duration", ColumnType.LONG) + .add("location", ColumnType.STRING) + .add("host", ColumnType.STRING) + .add("plaintext_port", ColumnType.LONG) + .add("tls_port", ColumnType.LONG) + .add("error_msg", ColumnType.STRING) .build(); static final RowSignature SUPERVISOR_SIGNATURE = RowSignature .builder() - .add("supervisor_id", ValueType.STRING) - .add("state", ValueType.STRING) - .add("detailed_state", ValueType.STRING) - .add("healthy", ValueType.LONG) - .add("type", ValueType.STRING) - .add("source", ValueType.STRING) - .add("suspended", ValueType.LONG) - .add("spec", ValueType.STRING) + .add("supervisor_id", ColumnType.STRING) + .add("state", ColumnType.STRING) + .add("detailed_state", ColumnType.STRING) + .add("healthy", ColumnType.LONG) + .add("type", ColumnType.STRING) + .add("source", ColumnType.STRING) + .add("suspended", ColumnType.LONG) + .add("spec", ColumnType.STRING) .build(); private final Map tableMap; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java index 91d19747b271..35fa3e46b9d0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/table/RowSignatures.java @@ -21,7 +21,9 @@ import com.google.common.base.Preconditions; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeComparability; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.sql.type.AbstractSqlType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; @@ -29,6 +31,7 @@ import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.ColumnHolder; +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.sql.calcite.expression.SimpleExtraction; @@ -57,7 +60,7 @@ public static RowSignature fromRelDataType(final List rowOrder, final Re for (int i = 0; i < rowOrder.size(); i++) { final RelDataType dataType = rowType.getFieldList().get(i).getType(); - final ValueType valueType = Calcites.getValueTypeForRelDataType(dataType); + final ColumnType valueType = Calcites.getColumnTypeForRelDataType(dataType); if (valueType == null) { throw new ISE( "Cannot translate sqlTypeName[%s] to Druid type for field[%s]", @@ -84,7 +87,7 @@ public static StringComparator getNaturalStringComparator( { Preconditions.checkNotNull(simpleExtraction, "simpleExtraction"); if (simpleExtraction.getExtractionFn() != null - || rowSignature.getColumnType(simpleExtraction.getColumn()).orElse(null) == ValueType.STRING) { + || rowSignature.getColumnType(simpleExtraction.getColumn()).map(type -> type.is(ValueType.STRING)).orElse(false)) { return StringComparators.LEXICOGRAPHIC; } else { return StringComparators.NUMERIC; @@ -104,11 +107,11 @@ public static RelDataType toRelDataType(final RowSignature rowSignature, final R if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { type = Calcites.createSqlType(typeFactory, SqlTypeName.TIMESTAMP); } else { - final ValueType columnType = + final ColumnType columnType = rowSignature.getColumnType(columnName) .orElseThrow(() -> new ISE("Encountered null type for column[%s]", columnName)); - switch (columnType) { + switch (columnType.getType()) { case STRING: // Note that there is no attempt here to handle multi-value in any special way. Maybe one day... type = Calcites.createSqlTypeWithNullability(typeFactory, SqlTypeName.VARCHAR, true); @@ -122,18 +125,26 @@ public static RelDataType toRelDataType(final RowSignature rowSignature, final R case DOUBLE: type = Calcites.createSqlTypeWithNullability(typeFactory, SqlTypeName.DOUBLE, nullNumeric); break; - case STRING_ARRAY: - type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.VARCHAR, true); - break; - case LONG_ARRAY: - type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.BIGINT, nullNumeric); - break; - case DOUBLE_ARRAY: - type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.DOUBLE, nullNumeric); + case ARRAY: + switch (columnType.getElementType().getType()) { + case STRING: + type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.VARCHAR, true); + break; + case LONG: + type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.BIGINT, nullNumeric); + break; + case DOUBLE: + type = Calcites.createSqlArrayTypeWithNullability(typeFactory, SqlTypeName.DOUBLE, nullNumeric); + break; + default: + throw new ISE("valueType[%s] not translatable", columnType); + } break; case COMPLEX: - // Loses information about exactly what kind of complex column this is. - type = Calcites.createSqlTypeWithNullability(typeFactory, SqlTypeName.OTHER, true); + type = typeFactory.createTypeWithNullability( + new ComplexSqlType(SqlTypeName.OTHER, columnType, true), + true + ); break; default: throw new ISE("valueType[%s] not translatable", columnType); @@ -145,4 +156,45 @@ public static RelDataType toRelDataType(final RowSignature rowSignature, final R return builder.build(); } + + /** + * Calcite {@link RelDataType} for Druid complex columns, to preserve complex type information + */ + public static final class ComplexSqlType extends AbstractSqlType + { + private final ColumnType columnType; + + public ComplexSqlType( + SqlTypeName typeName, + ColumnType columnType, + boolean isNullable + ) + { + super(typeName, isNullable, null); + this.columnType = columnType; + this.computeDigest(); + } + + @Override + public RelDataTypeComparability getComparability() + { + return RelDataTypeComparability.UNORDERED; + } + + @Override + protected void generateTypeString(StringBuilder sb, boolean withDetail) + { + sb.append(columnType.asTypeString()); + } + + public String getComplexTypeName() + { + return columnType.getComplexTypeName(); + } + + public String asTypeString() + { + return columnType.asTypeString(); + } + } } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 91d83fa86cbb..a51fe0a53d0a 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -612,7 +612,7 @@ public void testDatabaseMetaDataColumns() throws Exception Pair.of("TABLE_NAME", "foo"), Pair.of("COLUMN_NAME", "unique_dim1"), Pair.of("DATA_TYPE", Types.OTHER), - Pair.of("TYPE_NAME", "OTHER"), + Pair.of("TYPE_NAME", "COMPLEX"), Pair.of("IS_NULLABLE", "YES") ) ), @@ -695,7 +695,7 @@ public void testDatabaseMetaDataColumnsWithSuperuser() throws Exception Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE), Pair.of("COLUMN_NAME", "unique_dim1"), Pair.of("DATA_TYPE", Types.OTHER), - Pair.of("TYPE_NAME", "OTHER"), + Pair.of("TYPE_NAME", "COMPLEX"), Pair.of("IS_NULLABLE", "YES") ) ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 954c9fee8283..554cac824a03 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -63,7 +63,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQueryConfig; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.QueryStackTests; @@ -400,7 +400,7 @@ public static DimFilterHavingSpec having(final DimFilter filter) public static ExpressionVirtualColumn expressionVirtualColumn( final String name, final String expression, - final ValueType outputType + final ColumnType outputType ) { return new ExpressionVirtualColumn(name, expression, outputType, CalciteTests.createExprMacroTable()); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 78382b260029..55367fb16a5e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -44,7 +44,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.join.JoinType; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; @@ -71,7 +71,7 @@ public void testSelectConstantArrayExpressionFromTable() throws Exception newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "array(1,2)", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "array(1,2)", ColumnType.STRING)) .columns("dim1", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) @@ -97,7 +97,7 @@ public void testGroupByArrayFromCase() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "case_searched((\"dim4\" == 'a'),array('foo','bar','baz'),null)", - ValueType.STRING + ColumnType.STRING )) .setDimensions(new DefaultDimensionSpec("v0", "_d0")) .setGranularity(Granularities.ALL) @@ -123,7 +123,7 @@ public void testSelectNonConstantArrayExpressionFromTable() throws Exception newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "array(concat(\"dim1\",'word'),'up')", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "array(concat(\"dim1\",'word'),'up')", ColumnType.STRING)) .columns("dim1", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(5) @@ -149,10 +149,10 @@ public void testSelectNonConstantArrayExpressionFromTableFailForMultival() throw // // This error message will get better in the future. The error without translation would be: // - // org.apache.druid.java.util.common.RE: Unhandled array constructor element type [STRING_ARRAY] + // org.apache.druid.java.util.common.RE: Unhandled array constructor element type [ARRAY] expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Unhandled map function output type [STRING_ARRAY]"); + expectedException.expectMessage("Unhandled map function output type [ARRAY]"); testQuery( "SELECT ARRAY[CONCAT(dim3, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5", ImmutableList.of(), @@ -258,22 +258,22 @@ public void testSomeArrayFunctionsWithScanQuery() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( // these report as strings even though they are not, someday this will not be so - expressionVirtualColumn("v0", "array('a','b','c')", ValueType.STRING), - expressionVirtualColumn("v1", "array(1,2,3)", ValueType.STRING), - expressionVirtualColumn("v10", "array_concat(array(\"l1\"),array(\"l2\"))", ValueType.STRING), - expressionVirtualColumn("v11", "array_concat(array(\"d1\"),array(\"d2\"))", ValueType.STRING), - expressionVirtualColumn("v12", "array_offset(array(\"l1\"),0)", ValueType.STRING), - expressionVirtualColumn("v13", "array_offset(array(\"d1\"),0)", ValueType.STRING), - expressionVirtualColumn("v14", "array_ordinal(array(\"l1\"),1)", ValueType.STRING), - expressionVirtualColumn("v15", "array_ordinal(array(\"d1\"),1)", ValueType.STRING), - expressionVirtualColumn("v2", "array(1,2,4)", ValueType.STRING), - expressionVirtualColumn("v3", "array_append(\"dim3\",'foo')", ValueType.STRING), - expressionVirtualColumn("v4", "array_prepend('foo',array(\"dim2\"))", ValueType.STRING), - expressionVirtualColumn("v5", "array_append(array(1,2),\"l1\")", ValueType.STRING), - expressionVirtualColumn("v6", "array_prepend(\"l2\",array(1,2))", ValueType.STRING), - expressionVirtualColumn("v7", "array_append(array(1,2),\"d1\")", ValueType.STRING), - expressionVirtualColumn("v8", "array_prepend(\"d2\",array(1,2))", ValueType.STRING), - expressionVirtualColumn("v9", "array_concat(\"dim2\",\"dim3\")", ValueType.STRING) + expressionVirtualColumn("v0", "array('a','b','c')", ColumnType.STRING), + expressionVirtualColumn("v1", "array(1,2,3)", ColumnType.STRING), + expressionVirtualColumn("v10", "array_concat(array(\"l1\"),array(\"l2\"))", ColumnType.STRING), + expressionVirtualColumn("v11", "array_concat(array(\"d1\"),array(\"d2\"))", ColumnType.STRING), + expressionVirtualColumn("v12", "array_offset(array(\"l1\"),0)", ColumnType.STRING), + expressionVirtualColumn("v13", "array_offset(array(\"d1\"),0)", ColumnType.STRING), + expressionVirtualColumn("v14", "array_ordinal(array(\"l1\"),1)", ColumnType.STRING), + expressionVirtualColumn("v15", "array_ordinal(array(\"d1\"),1)", ColumnType.STRING), + expressionVirtualColumn("v2", "array(1,2,4)", ColumnType.STRING), + expressionVirtualColumn("v3", "array_append(\"dim3\",'foo')", ColumnType.STRING), + expressionVirtualColumn("v4", "array_prepend('foo',array(\"dim2\"))", ColumnType.STRING), + expressionVirtualColumn("v5", "array_append(array(1,2),\"l1\")", ColumnType.STRING), + expressionVirtualColumn("v6", "array_prepend(\"l2\",array(1,2))", ColumnType.STRING), + expressionVirtualColumn("v7", "array_append(array(1,2),\"d1\")", ColumnType.STRING), + expressionVirtualColumn("v8", "array_prepend(\"d2\",array(1,2))", ColumnType.STRING), + expressionVirtualColumn("v9", "array_concat(\"dim2\",\"dim3\")", ColumnType.STRING) ) .columns( "d1", @@ -385,18 +385,18 @@ public void testSomeArrayFunctionsWithScanQueryNoStringify() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( // these report as strings even though they are not, someday this will not be so - expressionVirtualColumn("v0", "array('a','b','c')", ValueType.STRING), - expressionVirtualColumn("v1", "array(1,2,3)", ValueType.STRING), - expressionVirtualColumn("v10", "array_concat(array(\"l1\"),array(\"l2\"))", ValueType.STRING), - expressionVirtualColumn("v11", "array_concat(array(\"d1\"),array(\"d2\"))", ValueType.STRING), - expressionVirtualColumn("v2", "array(1,2,4)", ValueType.STRING), - expressionVirtualColumn("v3", "array_append(\"dim3\",'foo')", ValueType.STRING), - expressionVirtualColumn("v4", "array_prepend('foo',array(\"dim2\"))", ValueType.STRING), - expressionVirtualColumn("v5", "array_append(array(1,2),\"l1\")", ValueType.STRING), - expressionVirtualColumn("v6", "array_prepend(\"l2\",array(1,2))", ValueType.STRING), - expressionVirtualColumn("v7", "array_append(array(1,2),\"d1\")", ValueType.STRING), - expressionVirtualColumn("v8", "array_prepend(\"d2\",array(1,2))", ValueType.STRING), - expressionVirtualColumn("v9", "array_concat(\"dim2\",\"dim3\")", ValueType.STRING) + expressionVirtualColumn("v0", "array('a','b','c')", ColumnType.STRING), + expressionVirtualColumn("v1", "array(1,2,3)", ColumnType.STRING), + expressionVirtualColumn("v10", "array_concat(array(\"l1\"),array(\"l2\"))", ColumnType.STRING), + expressionVirtualColumn("v11", "array_concat(array(\"d1\"),array(\"d2\"))", ColumnType.STRING), + expressionVirtualColumn("v2", "array(1,2,4)", ColumnType.STRING), + expressionVirtualColumn("v3", "array_append(\"dim3\",'foo')", ColumnType.STRING), + expressionVirtualColumn("v4", "array_prepend('foo',array(\"dim2\"))", ColumnType.STRING), + expressionVirtualColumn("v5", "array_append(array(1,2),\"l1\")", ColumnType.STRING), + expressionVirtualColumn("v6", "array_prepend(\"l2\",array(1,2))", ColumnType.STRING), + expressionVirtualColumn("v7", "array_append(array(1,2),\"d1\")", ColumnType.STRING), + expressionVirtualColumn("v8", "array_prepend(\"d2\",array(1,2))", ColumnType.STRING), + expressionVirtualColumn("v9", "array_concat(\"dim2\",\"dim3\")", ColumnType.STRING) ) .columns( "dim1", @@ -551,7 +551,7 @@ public void testArraySlice() throws Exception new Druids.ScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "array_slice(\"dim3\",1)", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "array_slice(\"dim3\",1)", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -582,11 +582,11 @@ public void testArrayLength() throws Exception .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_length(\"dim3\")", ValueType.LONG)) + .setVirtualColumns(expressionVirtualColumn("v0", "array_length(\"dim3\")", ColumnType.LONG)) .setDimensions( dimensions( - new DefaultDimensionSpec("dim1", "_d0", ValueType.STRING), - new DefaultDimensionSpec("v0", "_d1", ValueType.LONG) + new DefaultDimensionSpec("dim1", "_d0", ColumnType.STRING), + new DefaultDimensionSpec("v0", "_d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -649,11 +649,11 @@ public void testArrayAppend() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_append(\"dim3\",'foo')", - ValueType.STRING + ColumnType.STRING )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -709,11 +709,11 @@ public void testArrayPrepend() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_prepend('foo',\"dim3\")", - ValueType.STRING + ColumnType.STRING )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -766,18 +766,18 @@ public void testArrayPrependAppend() throws Exception expressionVirtualColumn( "v0", "array_to_string(array_prepend('foo',\"dim3\"),',')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "array_to_string(array_append(\"dim3\",'foo'),',')", - ValueType.STRING + ColumnType.STRING ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING), - new DefaultDimensionSpec("v1", "_d1", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING), + new DefaultDimensionSpec("v1", "_d1", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -831,11 +831,11 @@ public void testArrayConcat() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_concat(\"dim3\",\"dim3\")", - ValueType.STRING + ColumnType.STRING )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -867,10 +867,10 @@ public void testArrayOffset() throws Exception .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_offset(\"dim3\",1)", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "array_offset(\"dim3\",1)", ColumnType.STRING)) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -906,10 +906,10 @@ public void testArrayOrdinal() throws Exception .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal(\"dim3\",2)", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "array_ordinal(\"dim3\",2)", ColumnType.STRING)) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -948,11 +948,11 @@ public void testArrayOffsetOf() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_offset_of(\"dim3\",'b')", - ValueType.LONG + ColumnType.LONG )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) + new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -991,11 +991,11 @@ public void testArrayOrdinalOf() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_ordinal_of(\"dim3\",'b')", - ValueType.LONG + ColumnType.LONG )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) + new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1051,11 +1051,11 @@ public void testArrayToString() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "array_to_string(\"dim3\",',')", - ValueType.STRING + ColumnType.STRING )) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1107,17 +1107,17 @@ public void testArrayToStringToMultiValueString() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "array_length(\"dim3\")", ValueType.LONG), + expressionVirtualColumn("v0", "array_length(\"dim3\")", ColumnType.LONG), expressionVirtualColumn( "v1", "string_to_array(concat(array_to_string(\"dim3\",','),',d'),',')", - ValueType.STRING + ColumnType.STRING ) ) .setDimFilter(bound("v0", "0", null, true, false, null, StringComparators.NUMERIC)) .setDimensions( dimensions( - new DefaultDimensionSpec("v1", "_d0", ValueType.STRING) + new DefaultDimensionSpec("v1", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1442,7 +1442,7 @@ public void testArrayAggExpression() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "concat(\"dim1\",\"dim2\")", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"dim1\",\"dim2\")", ColumnType.STRING) ) .aggregators( aggregators( @@ -1592,7 +1592,7 @@ public void testArrayAggAsArrayFromJoin() throws Exception ) ) .virtualColumns( - expressionVirtualColumn("v0", "array_to_string(\"j0.a0\",',')", ValueType.STRING) + expressionVirtualColumn("v0", "array_to_string(\"j0.a0\",',')", ColumnType.STRING) ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim4", "j0.a0", "v0") @@ -1612,7 +1612,7 @@ public void testArrayAggGroupByArrayAggFromSubquery() throws Exception cannotVectorize(); // yo, can't group on array types right now so expect failure expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Cannot create query type helper from invalid type [STRING_ARRAY]"); + expectedException.expectMessage("Cannot create query type helper from invalid type [ARRAY]"); testQuery( "SELECT dim2, arr, COUNT(*) FROM (SELECT dim2, ARRAY_AGG(DISTINCT dim1) as arr FROM foo WHERE dim1 is not null GROUP BY 1 LIMIT 5) GROUP BY 1,2", ImmutableList.of(), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java index 6d4bf1bc4594..2ce8842242d0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java @@ -40,7 +40,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -87,7 +87,7 @@ public void testCorrelatedSubquery(Map queryContext) throws Exce .setVirtualColumns(new ExpressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE )) .setDimFilter(not(selector("country", null, null))) @@ -95,7 +95,7 @@ public void testCorrelatedSubquery(Map queryContext) throws Exce new DefaultDimensionSpec( "v0", "d0", - ValueType.LONG + ColumnType.LONG ), new DefaultDimensionSpec( "country", @@ -200,7 +200,7 @@ public void testCorrelatedSubqueryWithLeftFilter(Map queryContex .setVirtualColumns(new ExpressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE )) .setDimFilter(not(selector("country", null, null))) @@ -208,7 +208,7 @@ public void testCorrelatedSubqueryWithLeftFilter(Map queryContex new DefaultDimensionSpec( "v0", "d0", - ValueType.LONG + ColumnType.LONG ), new DefaultDimensionSpec( "country", @@ -293,7 +293,7 @@ public void testCorrelatedSubqueryWithLeftFilter_leftDirectAccessDisabled(Map .setVirtualColumns(new ExpressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG, + ColumnType.LONG, TestExprMacroTable.INSTANCE )) .setDimensions( new DefaultDimensionSpec( "v0", "d0", - ValueType.LONG + ColumnType.LONG ), new DefaultDimensionSpec( "country", @@ -473,14 +473,14 @@ public void testCorrelatedSubqueryWithCorrelatedQueryFilter_Scan(Map array_contains(array('b'), x), \"dim3\"))", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) + new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1099,12 +1099,12 @@ public void testMultiValueListFilterComposedDeny() throws Exception expressionVirtualColumn( "v0", "array_length(filter((x) -> !array_contains(array('b'), x), \"dim3\"))", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.LONG) + new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1147,7 +1147,7 @@ public void testFilterOnMultiValueListFilterNoMatch() throws Exception .setDimFilter(selector("v0", "a", null)) .setDimensions( dimensions( - new DefaultDimensionSpec("dim3", "_d0", ValueType.STRING) + new DefaultDimensionSpec("dim3", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1190,7 +1190,7 @@ public void testFilterOnMultiValueListFilterMatch() throws Exception .setDimFilter(selector("v0", "b", null)) .setDimensions( dimensions( - new DefaultDimensionSpec("dim3", "_d0", ValueType.STRING) + new DefaultDimensionSpec("dim3", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -1237,7 +1237,7 @@ public void testFilterOnMultiValueListFilterMatchLike() throws Exception .setDimFilter(new LikeDimFilter("v0", "b%", null, null)) .setDimensions( dimensions( - new DefaultDimensionSpec("dim3", "_d0", ValueType.STRING) + new DefaultDimensionSpec("dim3", "_d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index f5512b5e8fb7..d069224f4b02 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -36,8 +36,8 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQuery.ResultFormat; +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.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.http.SqlParameter; @@ -63,7 +63,7 @@ public void testSelectConstantParamGetsConstant() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{2L}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -88,7 +88,7 @@ public void testParamsGetOptimizedIntoConstant() throws Exception newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "2", ValueType.LONG)) + .virtualColumns(expressionVirtualColumn("v0", "2", ColumnType.LONG)) .columns("dim1", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) @@ -238,7 +238,7 @@ public void testParamsInSelectExpressionAndLimit() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "substring(\"dim2\", 0, 1)", ValueType.STRING) + expressionVirtualColumn("v0", "substring(\"dim2\", 0, 1)", ColumnType.STRING) ) .columns("v0") .limit(2) @@ -271,7 +271,7 @@ public void testParamsTuckedInACast() throws Exception .setDimFilter(expressionFilter("((\"m1\" - 1) == CAST(\"dim1\", 'DOUBLE'))")) .setDimensions(dimensions( new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) )) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) @@ -634,7 +634,7 @@ public void testWrongTypeParameter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("f1", ValueType.FLOAT).add("l1", ValueType.LONG).build() + RowSignature.builder().add("f1", ColumnType.FLOAT).add("l1", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -666,10 +666,10 @@ public void testNullParameter() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'parameter')", - ValueType.STRING + ColumnType.STRING ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -701,10 +701,10 @@ public void testNullParameter() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'parameter')", - ValueType.STRING + ColumnType.STRING ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 7478988aaa9c..17724c440709 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -112,8 +112,8 @@ import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; +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.join.JoinType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.QueryLifecycle; @@ -162,7 +162,7 @@ public void testSelectConstantExpression() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("ZERO", ValueType.LONG).build() + RowSignature.builder().add("ZERO", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -170,7 +170,7 @@ public void testSelectConstantExpression() throws Exception new ExpressionVirtualColumn( "v0", "'f'", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ) ) @@ -196,7 +196,7 @@ public void testExpressionContainingNull() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("ZERO", ValueType.LONG).build() + RowSignature.builder().add("ZERO", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -204,7 +204,7 @@ public void testExpressionContainingNull() throws Exception new ExpressionVirtualColumn( "v0", "array('Hello',null)", - ValueType.STRING, + ColumnType.STRING, ExprMacroTable.nil() ) ) @@ -237,11 +237,11 @@ public void testSelectNonNumericNumberLiterals() throws Exception new Object[]{Long.MAX_VALUE, Long.MAX_VALUE, Long.MIN_VALUE, Long.MIN_VALUE, 0L} ), RowSignature.builder() - .add("EXPR$0", ValueType.LONG) - .add("EXPR$1", ValueType.LONG) - .add("EXPR$2", ValueType.LONG) - .add("EXPR$3", ValueType.LONG) - .add("EXPR$4", ValueType.LONG) + .add("EXPR$0", ColumnType.LONG) + .add("EXPR$1", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) + .add("EXPR$3", ColumnType.LONG) + .add("EXPR$4", ColumnType.LONG) .build() ) ) @@ -272,7 +272,7 @@ public void testSelectConstantExpressionFromTable() throws Exception newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "2", ValueType.LONG)) + .virtualColumns(expressionVirtualColumn("v0", "2", ColumnType.LONG)) .columns("dim1", "v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) @@ -338,7 +338,7 @@ public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries( expressionVirtualColumn( "v0", "timestamp_format(\"d0\",'yyyy-MM','UTC')", - ValueType.STRING + ColumnType.STRING ) ) .setGranularity(Granularities.ALL) @@ -442,7 +442,7 @@ public void testJoinOuterGroupByAndSubqueryHasLimit() throws Exception .build() ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs( useDefault @@ -525,7 +525,7 @@ public void testJoinOuterGroupByAndSubqueryNoLimit(Map queryCont ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs( useDefault @@ -612,7 +612,7 @@ public void testJoinWithLimitBeforeJoining() throws Exception ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs( useDefault @@ -695,8 +695,8 @@ public void testJoinOnTimeseriesWithFloorOnTime() throws Exception ) .setInterval(querySegmentSpec(Filtration.eternity())) .setDimensions( - new DefaultDimensionSpec("__time", "d0", ValueType.LONG), - new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + new DefaultDimensionSpec("__time", "d0", ColumnType.LONG), + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) ) .setGranularity(Granularities.ALL) @@ -739,12 +739,12 @@ public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() throws Excepti expressionVirtualColumn( "v0", "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 0)", - ValueType.LONG + ColumnType.LONG ) ) .setDimFilter(selector("dim3", "b", null)) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators( new FloatMinAggregatorFactory("a0", "m1") )) @@ -757,8 +757,8 @@ public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() throws Excepti ) .setInterval(querySegmentSpec(Filtration.eternity())) .setDimensions( - new DefaultDimensionSpec("__time", "d0", ValueType.LONG), - new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + new DefaultDimensionSpec("__time", "d0", ColumnType.LONG), + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) ) .setGranularity(Granularities.ALL) @@ -990,14 +990,14 @@ public void testBitwiseExpressions() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .columns("v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7") .virtualColumns( - expressionVirtualColumn("v0", "bitwiseAnd(\"l1\",\"l2\")", ValueType.LONG), - expressionVirtualColumn("v1", "bitwiseOr(\"l1\",\"l2\")", ValueType.LONG), - expressionVirtualColumn("v2", "bitwiseXor(\"l1\",\"l2\")", ValueType.LONG), - expressionVirtualColumn("v3", "bitwiseComplement(\"l1\")", ValueType.LONG), - expressionVirtualColumn("v4", "bitwiseShiftLeft(\"l1\",2)", ValueType.LONG), - expressionVirtualColumn("v5", "bitwiseShiftRight(\"l1\",2)", ValueType.LONG), - expressionVirtualColumn("v6", "bitwiseConvertDoubleToLongBits(\"d1\")", ValueType.LONG), - expressionVirtualColumn("v7", "bitwiseConvertLongBitsToDouble(\"l1\")", ValueType.DOUBLE) + expressionVirtualColumn("v0", "bitwiseAnd(\"l1\",\"l2\")", ColumnType.LONG), + expressionVirtualColumn("v1", "bitwiseOr(\"l1\",\"l2\")", ColumnType.LONG), + expressionVirtualColumn("v2", "bitwiseXor(\"l1\",\"l2\")", ColumnType.LONG), + expressionVirtualColumn("v3", "bitwiseComplement(\"l1\")", ColumnType.LONG), + expressionVirtualColumn("v4", "bitwiseShiftLeft(\"l1\",2)", ColumnType.LONG), + expressionVirtualColumn("v5", "bitwiseShiftRight(\"l1\",2)", ColumnType.LONG), + expressionVirtualColumn("v6", "bitwiseConvertDoubleToLongBits(\"d1\")", ColumnType.LONG), + expressionVirtualColumn("v7", "bitwiseConvertLongBitsToDouble(\"l1\")", ColumnType.DOUBLE) ) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -1130,7 +1130,7 @@ public void testInformationSchemaColumnsOnTable() throws Exception new Object[]{"dim3", "VARCHAR", "YES"}, new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"}, new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"}, - new Object[]{"unique_dim1", "OTHER", "YES"} + new Object[]{"unique_dim1", "COMPLEX", "YES"} ) ); } @@ -1160,7 +1160,7 @@ public void testInformationSchemaColumnsOnForbiddenTable() throws Exception new Object[]{"dim2", "VARCHAR", "YES"}, new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"}, new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"}, - new Object[]{"unique_dim1", "OTHER", "YES"} + new Object[]{"unique_dim1", "COMPLEX", "YES"} ) ); } @@ -1326,8 +1326,8 @@ public void testSelectStarOnForbiddenView() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ValueType.STRING), - expressionVirtualColumn("v1", "'a'", ValueType.STRING) + expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING), + expressionVirtualColumn("v1", "'a'", ColumnType.STRING) ) .filters(selector("dim2", "a", null)) .columns("__time", "v0", "v1") @@ -1433,7 +1433,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n", + "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n", "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]" } ) @@ -1501,7 +1501,7 @@ public void testSelectWithProjection() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "substring(\"dim2\", 0, 1)", ValueType.STRING) + expressionVirtualColumn("v0", "substring(\"dim2\", 0, 1)", ColumnType.STRING) ) .columns("v0") .limit(2) @@ -1526,7 +1526,7 @@ public void testSelectWithExpressionFilter() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "(\"m1\" + 1)", ValueType.FLOAT) + expressionVirtualColumn("v0", "(\"m1\" + 1)", ColumnType.FLOAT) ) .filters(selector("v0", "7", null)) .columns("dim1") @@ -1937,7 +1937,7 @@ public void testTopNLimitWrapping() throws Exception new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("dim1", "d0", ValueType.STRING)) + .dimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) .threshold(2) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .metric( @@ -1963,7 +1963,7 @@ public void testTopNLimitWrappingOrderByAgg() throws Exception new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("dim1", "d0", ValueType.STRING)) + .dimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) .threshold(2) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .metric("a0") @@ -1998,8 +1998,8 @@ public void testGroupByLimitWrapping() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions( - new DefaultDimensionSpec("dim1", "d0", ValueType.STRING), - new DefaultDimensionSpec("dim2", "d1", ValueType.STRING) + new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) ) .setLimitSpec( DefaultLimitSpec @@ -2031,8 +2031,8 @@ public void testGroupByWithForceLimitPushDown() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions( - new DefaultDimensionSpec("dim1", "d0", ValueType.STRING), - new DefaultDimensionSpec("dim2", "d1", ValueType.STRING) + new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) ) .setLimitSpec( new DefaultLimitSpec( @@ -2060,8 +2060,8 @@ public void testGroupByLimitWrappingOrderByAgg() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions( - new DefaultDimensionSpec("dim1", "d0", ValueType.STRING), - new DefaultDimensionSpec("dim2", "d1", ValueType.STRING) + new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING), + new DefaultDimensionSpec("dim2", "d1", ColumnType.STRING) ) .setLimitSpec( DefaultLimitSpec @@ -2090,7 +2090,7 @@ public void testSelectProjectionFromSelectSingleColumnWithInnerLimitDescending() newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .limit(2) .order(ScanQuery.Order.DESCENDING) @@ -2119,7 +2119,7 @@ public void testSelectProjectionFromSelectSingleColumnDescending() throws Except newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .order(ScanQuery.Order.NONE) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2146,7 +2146,7 @@ public void testSelectProjectionFromSelectSingleColumnWithInnerAndOuterLimitDesc newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat('beep ',\"dim1\")", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .limit(2) .order(ScanQuery.Order.DESCENDING) @@ -2217,9 +2217,9 @@ public void testEarliestAggregators() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "(\"cnt\" + 1)", ValueType.LONG), - expressionVirtualColumn("v1", "(\"m1\" + 1)", ValueType.FLOAT), - expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ValueType.STRING) + expressionVirtualColumn("v0", "(\"cnt\" + 1)", ColumnType.LONG), + expressionVirtualColumn("v1", "(\"m1\" + 1)", ColumnType.FLOAT), + expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ColumnType.STRING) ) .aggregators( aggregators( @@ -2257,9 +2257,9 @@ public void testLatestAggregators() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "(\"cnt\" + 1)", ValueType.LONG), - expressionVirtualColumn("v1", "(\"m1\" + 1)", ValueType.FLOAT), - expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ValueType.STRING) + expressionVirtualColumn("v0", "(\"cnt\" + 1)", ColumnType.LONG), + expressionVirtualColumn("v1", "(\"m1\" + 1)", ColumnType.FLOAT), + expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ColumnType.STRING) ) .aggregators( aggregators( @@ -2298,9 +2298,9 @@ public void testAnyAggregator() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "(\"cnt\" + 1)", ValueType.LONG), - expressionVirtualColumn("v1", "(\"m1\" + 1)", ValueType.FLOAT), - expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ValueType.STRING) + expressionVirtualColumn("v0", "(\"cnt\" + 1)", ColumnType.LONG), + expressionVirtualColumn("v1", "(\"m1\" + 1)", ColumnType.FLOAT), + expressionVirtualColumn("v2", "concat(\"dim1\",CAST(\"cnt\", 'STRING'))", ColumnType.STRING) ) .aggregators( aggregators( @@ -3279,7 +3279,7 @@ public void testGroupByLong() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3300,7 +3300,7 @@ public void testGroupByOrdinal() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3322,7 +3322,7 @@ public void testGroupByAndOrderByAlias() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( DefaultLimitSpec @@ -3380,7 +3380,7 @@ public void testGroupByAndOrderByOrdinalOfAlias() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( DefaultLimitSpec @@ -3413,7 +3413,7 @@ public void testGroupByFloat() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT))) + .setDimensions(dimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3439,7 +3439,7 @@ public void testGroupByDouble() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE))) + .setDimensions(dimensions(new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3574,7 +3574,7 @@ public void testHavingOnApproximateCountDistinct() throws Exception "a0", null, ImmutableList.of( - new DefaultDimensionSpec("m1", "m1", ValueType.FLOAT) + new DefaultDimensionSpec("m1", "m1", ColumnType.FLOAT) ), false, true @@ -3626,8 +3626,8 @@ public void testHavingOnExactCountDistinct() throws Exception .setGranularity(Granularities.ALL) .setDimensions( dimensions( - new DefaultDimensionSpec("dim2", "d0", ValueType.STRING), - new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING), + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) ) ) .setContext(QUERY_CONTEXT_DEFAULT) @@ -3636,7 +3636,7 @@ public void testHavingOnExactCountDistinct() throws Exception ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING))) .setAggregatorSpecs( aggregators( useDefault @@ -3727,7 +3727,7 @@ public void testColumnComparison() throws Exception .setDimFilter(expressionFilter("((\"m1\" - 1) == CAST(\"dim1\", 'DOUBLE'))")) .setDimensions(dimensions( new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + new DefaultDimensionSpec("m1", "d1", ColumnType.FLOAT) )) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) @@ -4697,7 +4697,7 @@ public void testGroupByCaseWhen() throws Exception + "(timestamp_extract(\"__time\",'DAY','UTC') == 0)," + "'zero'," + DruidExpression.nullLiteral() + ")", - ValueType.STRING + ColumnType.STRING ) ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0"))) @@ -4734,7 +4734,7 @@ public void testGroupByCaseWhenOfTripleAnd() throws Exception expressionVirtualColumn( "v0", "case_searched(((\"m1\" > 1) && (\"m1\" < 5) && (\"cnt\" == 1)),'x',null)", - ValueType.STRING + ColumnType.STRING ) ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0"))) @@ -4799,7 +4799,7 @@ public void testNullLongFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -4837,7 +4837,7 @@ public void testNullDoubleFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -4876,7 +4876,7 @@ public void testNullFloatFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -4926,7 +4926,7 @@ public void testNullDoubleTopN() throws Exception new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("d1", "_d0", ValueType.DOUBLE)) + .dimension(new DefaultDimensionSpec("d1", "_d0", ColumnType.DOUBLE)) .threshold(10) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .metric( @@ -4966,7 +4966,7 @@ public void testNullFloatTopN() throws Exception new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("f1", "_d0", ValueType.FLOAT)) + .dimension(new DefaultDimensionSpec("f1", "_d0", ColumnType.FLOAT)) .threshold(10) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .metric( @@ -5006,7 +5006,7 @@ public void testNullLongTopN() throws Exception new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("l1", "_d0", ValueType.LONG)) + .dimension(new DefaultDimensionSpec("l1", "_d0", ColumnType.LONG)) .threshold(10) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .metric( @@ -5032,7 +5032,7 @@ public void testLongPredicateIsNull() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .columns("v0") .virtualColumns( - expressionVirtualColumn("v0", NullHandling.replaceWithDefault() ? "0" : "isnull(\"l1\")", ValueType.LONG) + expressionVirtualColumn("v0", NullHandling.replaceWithDefault() ? "0" : "isnull(\"l1\")", ColumnType.LONG) ) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -5178,7 +5178,7 @@ public void testNullStringEquality() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -5211,10 +5211,10 @@ public void testCoalesceColumns() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",\"dim1\")", - ValueType.STRING + ColumnType.STRING ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -5349,7 +5349,7 @@ public void testGroupingWithNullInFilter() throws Exception new TableDataSource(CalciteTests.DATASOURCE1), InlineDataSource.fromIterable( ImmutableList.of(new Object[]{null}), - RowSignature.builder().add("ROW_VALUE", ValueType.STRING).build() + RowSignature.builder().add("ROW_VALUE", ColumnType.STRING).build() ), "j0.", "(\"dim1\" == \"j0.ROW_VALUE\")", @@ -5388,7 +5388,7 @@ public void testTwoExactCountDistincts() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim1", "d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) .setContext(QUERY_CONTEXT_DEFAULT) .build() ) @@ -5412,7 +5412,7 @@ public void testTwoExactCountDistincts() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) .setContext(QUERY_CONTEXT_DEFAULT) .build() ) @@ -5507,7 +5507,7 @@ public void testGroupByNothingWithLiterallyFalseFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(new Object[]{0L, null}), - RowSignature.builder().add("EXPR$0", ValueType.LONG).add("EXPR$1", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).add("EXPR$1", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -5579,7 +5579,7 @@ public void testGroupByOneColumnWithLiterallyFalseFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("EXPR$0", ValueType.LONG).add("EXPR$1", ValueType.LONG).build() + RowSignature.builder().add("EXPR$0", ColumnType.LONG).add("EXPR$1", ColumnType.LONG).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -6091,7 +6091,7 @@ public void testFilterOnStringAsNumber() throws Exception expressionVirtualColumn( "v0", "floor(CAST(\"dim1\", 'DOUBLE'))", - ValueType.DOUBLE + ColumnType.DOUBLE ) ) .setDimFilter( @@ -6479,7 +6479,7 @@ public void testFilteredAggregations() throws Exception new CardinalityAggregatorFactory( "a10", null, - dimensions(new DefaultDimensionSpec("m1", "m1", ValueType.FLOAT)), + dimensions(new DefaultDimensionSpec("m1", "m1", ColumnType.FLOAT)), false, true ), @@ -6517,7 +6517,7 @@ public void testCaseFilteredAggregationWithGroupBy() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), @@ -6634,10 +6634,10 @@ public void testExpressionFilteringAndGrouping() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "(floor((\"m1\" / 2)) * 2)", ValueType.FLOAT) + expressionVirtualColumn("v0", "(floor((\"m1\" / 2)) * 2)", ColumnType.FLOAT) ) .setDimFilter(bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC)) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.FLOAT))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( DefaultLimitSpec @@ -6680,12 +6680,12 @@ public void testExpressionFilteringAndGroupingUsingCastToLong() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "((CAST(\"m1\", 'LONG') / 2) * 2)", ValueType.LONG) + expressionVirtualColumn("v0", "((CAST(\"m1\", 'LONG') / 2) * 2)", ColumnType.LONG) ) .setDimFilter( bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( new DefaultLimitSpec( @@ -6731,13 +6731,13 @@ public void testExpressionFilteringAndGroupingOnStringCastToNumber() throws Exce expressionVirtualColumn( "v0", "(floor((CAST(\"dim1\", 'DOUBLE') / 2)) * 2)", - ValueType.FLOAT + ColumnType.FLOAT ) ) .setDimFilter( bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.FLOAT))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( new DefaultLimitSpec( @@ -6870,7 +6870,7 @@ public void testCountStarWithNotOfDegenerateFilter() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("dim1", ValueType.STRING).add("dim2", ValueType.STRING).build() + RowSignature.builder().add("dim1", ColumnType.STRING).add("dim2", ColumnType.STRING).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -7491,7 +7491,7 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() thr .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "timestamp_floor(\"cnt\",'P1Y',null,'UTC')", ValueType.LONG) + expressionVirtualColumn("v0", "timestamp_floor(\"cnt\",'P1Y',null,'UTC')", ColumnType.LONG) ) .setDimFilter( bound( @@ -7504,7 +7504,7 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() thr StringComparators.NUMERIC ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -7565,12 +7565,12 @@ public void testSelectDistinctWithStrlenFilter() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "strlen(\"dim1\")", ValueType.LONG), + expressionVirtualColumn("v0", "strlen(\"dim1\")", ColumnType.LONG), // The two layers of CASTs here are unusual, they should really be collapsed into one expressionVirtualColumn( "v1", "CAST(CAST(strlen(\"dim1\"), 'STRING'), 'LONG')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) @@ -7728,7 +7728,7 @@ public void testSelectNonAggregatingWithLimitLiterallyZero() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("dim2", ValueType.STRING).build() + RowSignature.builder().add("dim2", ColumnType.STRING).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -7754,7 +7754,7 @@ public void testSelectNonAggregatingWithLimitReducedToZero() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("dim2", ValueType.STRING).build() + RowSignature.builder().add("dim2", ColumnType.STRING).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -7779,7 +7779,7 @@ public void testSelectAggregatingWithLimitReducedToZero() throws Exception .dataSource( InlineDataSource.fromIterable( ImmutableList.of(), - RowSignature.builder().add("dim2", ValueType.STRING).build() + RowSignature.builder().add("dim2", ColumnType.STRING).build() ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -7844,7 +7844,7 @@ public void testCountDistinctOfCaseWhen() throws Exception new CardinalityAggregatorFactory( "a0", null, - ImmutableList.of(new DefaultDimensionSpec("m1", "m1", ValueType.FLOAT)), + ImmutableList.of(new DefaultDimensionSpec("m1", "m1", ColumnType.FLOAT)), false, true ), @@ -7854,7 +7854,7 @@ public void testCountDistinctOfCaseWhen() throws Exception new CardinalityAggregatorFactory( "a1", null, - ImmutableList.of(new DefaultDimensionSpec("dim1", "dim1", ValueType.STRING)), + ImmutableList.of(new DefaultDimensionSpec("dim1", "dim1", ColumnType.STRING)), false, true ), @@ -8021,10 +8021,10 @@ public void testMultipleExactCountDistinctWithGroupingAndOtherAggregators() thro .setVirtualColumns(expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG + ColumnType.LONG )) .setDimensions(dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("city", "d1"), new DefaultDimensionSpec("user", "d2") )) @@ -8051,7 +8051,7 @@ public void testMultipleExactCountDistinctWithGroupingAndOtherAggregators() thro ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("d0", "_d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("_a0"), @@ -8094,7 +8094,7 @@ public void testApproxCountDistinct() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "concat(substring(\"dim2\", 0, 1),'x')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(substring(\"dim2\", 0, 1),'x')", ColumnType.STRING) ) .aggregators( aggregators( @@ -8123,7 +8123,7 @@ public void testApproxCountDistinct() throws Exception new ExtractionDimensionSpec( "dim2", "dim2", - ValueType.STRING, + ColumnType.STRING, new SubstringDimExtractionFn(0, 1) ) ), @@ -8133,7 +8133,7 @@ public void testApproxCountDistinct() throws Exception new CardinalityAggregatorFactory( "a4", null, - dimensions(new DefaultDimensionSpec("v0", "v0", ValueType.STRING)), + dimensions(new DefaultDimensionSpec("v0", "v0", ColumnType.STRING)), false, true ), @@ -8164,7 +8164,7 @@ public void testApproxCountDistinctOnVectorizableSingleStringExpression() throws .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "concat(\"dim1\",'hello')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"dim1\",'hello')", ColumnType.STRING) ) .aggregators( aggregators( @@ -8212,7 +8212,7 @@ public void testNestedGroupBy() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions( - new DefaultDimensionSpec("m2", "d0", ValueType.DOUBLE), + new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE), new DefaultDimensionSpec("dim1", "d1") )) .setDimFilter(new SelectorDimFilter("m1", "5.0", null)) @@ -8226,12 +8226,12 @@ public void testNestedGroupBy() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"a0\",'PT1H',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions(dimensions( - new DefaultDimensionSpec("v0", "_d0", ValueType.LONG), - new DefaultDimensionSpec("d1", "_d1", ValueType.STRING) + new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG), + new DefaultDimensionSpec("d1", "_d1", ColumnType.STRING) )) .setAggregatorSpecs( aggregators( @@ -8549,7 +8549,7 @@ public void testAvgDailyCountDistinct() throws Exception dimensions(new DefaultDimensionSpec( "cnt", "cnt", - ValueType.LONG + ColumnType.LONG )), false, true @@ -8730,7 +8730,7 @@ public void testTopNFilterJoinWithProjection(Map queryContext) t new ExtractionDimensionSpec( "dim1", "d0", - ValueType.STRING, + ColumnType.STRING, new SubstringDimExtractionFn(0, 10) ) ) @@ -8899,7 +8899,7 @@ public void testExplainExactCountDistinctOfSemiJoinResult() throws Exception final String explanation = "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" + " DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":null,\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null,\"extractionFn\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"descending\":false}], signature=[{d0:STRING}])\n"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; @@ -9295,14 +9295,14 @@ public void testCountDistinctOfTrim() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .virtualColumns(expressionVirtualColumn("v0", "trim(\"dim1\",' ')", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "trim(\"dim1\",' ')", ColumnType.STRING)) .filters(not(selector("v0", NullHandling.emptyToNullIfNeeded(""), null))) .aggregators( aggregators( new CardinalityAggregatorFactory( "a0", null, - dimensions(new DefaultDimensionSpec("v0", "v0", ValueType.STRING)), + dimensions(new DefaultDimensionSpec("v0", "v0", ColumnType.STRING)), false, true ) @@ -9337,9 +9337,9 @@ public void testSillyQuarters() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "(((timestamp_extract(\"__time\",'MONTH','UTC') - 1) / 3) + 1)", - ValueType.LONG + ColumnType.LONG )) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -9417,7 +9417,7 @@ public void testRegexpExtractFilterViaNotNullCheck() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "regexp_extract(concat('Z',\"dim1\"),'^Z2')", ValueType.STRING) + expressionVirtualColumn("v0", "regexp_extract(concat('Z',\"dim1\"),'^Z2')", ColumnType.STRING) ) .filters( or( @@ -9448,7 +9448,7 @@ public void testRegexpLikeFilter() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "concat('Z',\"dim1\")", ValueType.STRING) + expressionVirtualColumn("v0", "concat('Z',\"dim1\")", ColumnType.STRING) ) .filters( or( @@ -9579,7 +9579,7 @@ public void testGroupByLimitPushdownExtraction() throws Exception new ExtractionDimensionSpec("dim5", "_d1", new SubstringDimExtractionFn(0, 1)) ) ) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimFilter(selector("dim4", "a", null)) .setAggregatorSpecs( aggregators( @@ -9706,9 +9706,9 @@ public void testSelectCurrentTimeAndDateLosAngeles() throws Exception } ), RowSignature.builder() - .add("CURRENT_TIMESTAMP", ValueType.LONG) - .add("CURRENT_DATE", ValueType.LONG) - .add("EXPR$2", ValueType.LONG) + .add("CURRENT_TIMESTAMP", ColumnType.LONG) + .add("CURRENT_DATE", ColumnType.LONG) + .add("EXPR$2", ColumnType.LONG) .build() ) ) @@ -9880,8 +9880,8 @@ public void testFilterOnTimeExtract() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ValueType.LONG), - expressionVirtualColumn("v1", "timestamp_extract(\"__time\",'MONTH','UTC')", ValueType.LONG) + expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ColumnType.LONG), + expressionVirtualColumn("v1", "timestamp_extract(\"__time\",'MONTH','UTC')", ColumnType.LONG) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .filters( @@ -9918,12 +9918,12 @@ public void testFilterOnTimeExtractWithMultipleDays() throws Exception expressionVirtualColumn( "v0", "timestamp_extract(\"__time\",'YEAR','UTC')", - ValueType.LONG + ColumnType.LONG ), expressionVirtualColumn( "v1", "timestamp_extract(\"__time\",'DAY','UTC')", - ValueType.LONG + ColumnType.LONG ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -9965,22 +9965,22 @@ public void testFilterOnTimeExtractWithVariousTimeUnits() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ValueType.LONG), + expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'YEAR','UTC')", ColumnType.LONG), expressionVirtualColumn( "v1", "timestamp_extract(\"__time\",'MICROSECOND','UTC')", - ValueType.LONG + ColumnType.LONG ), expressionVirtualColumn( "v2", "timestamp_extract(\"__time\",'MILLISECOND','UTC')", - ValueType.LONG + ColumnType.LONG ), - expressionVirtualColumn("v3", "timestamp_extract(\"__time\",'ISODOW','UTC')", ValueType.LONG), - expressionVirtualColumn("v4", "timestamp_extract(\"__time\",'ISOYEAR','UTC')", ValueType.LONG), - expressionVirtualColumn("v5", "timestamp_extract(\"__time\",'DECADE','UTC')", ValueType.LONG), - expressionVirtualColumn("v6", "timestamp_extract(\"__time\",'CENTURY','UTC')", ValueType.LONG), - expressionVirtualColumn("v7", "timestamp_extract(\"__time\",'MILLENNIUM','UTC')", ValueType.LONG) + expressionVirtualColumn("v3", "timestamp_extract(\"__time\",'ISODOW','UTC')", ColumnType.LONG), + expressionVirtualColumn("v4", "timestamp_extract(\"__time\",'ISOYEAR','UTC')", ColumnType.LONG), + expressionVirtualColumn("v5", "timestamp_extract(\"__time\",'DECADE','UTC')", ColumnType.LONG), + expressionVirtualColumn("v6", "timestamp_extract(\"__time\",'CENTURY','UTC')", ColumnType.LONG), + expressionVirtualColumn("v7", "timestamp_extract(\"__time\",'MILLENNIUM','UTC')", ColumnType.LONG) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .filters( @@ -10034,9 +10034,9 @@ public void testGroupByFloor() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "floor(CAST(\"dim1\", 'DOUBLE'))", ValueType.FLOAT) + expressionVirtualColumn("v0", "floor(CAST(\"dim1\", 'DOUBLE'))", ColumnType.FLOAT) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.FLOAT))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -10064,7 +10064,7 @@ public void testGroupByFloorWithOrderBy() throws Exception expressionVirtualColumn( "v0", "floor(CAST(\"dim1\", 'DOUBLE'))", - ValueType.FLOAT + ColumnType.FLOAT ) ) .setDimensions( @@ -10072,7 +10072,7 @@ public void testGroupByFloorWithOrderBy() throws Exception new DefaultDimensionSpec( "v0", "d0", - ValueType.FLOAT + ColumnType.FLOAT ) ) ) @@ -10118,12 +10118,12 @@ public void testGroupByFloorTimeAndOneOtherDimensionWithOrderBy() throws Excepti expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1Y',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("dim2", "d1") ) ) @@ -10189,8 +10189,8 @@ public void testGroupByStringLength() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "strlen(\"dim1\")", ValueType.LONG)) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setVirtualColumns(expressionVirtualColumn("v0", "strlen(\"dim1\")", ColumnType.LONG)) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -10240,7 +10240,7 @@ public void testFilterAndGroupByLookup() throws Exception new ExtractionDimensionSpec( "dim1", "d0", - ValueType.STRING, + ColumnType.STRING, extractionFn ) ) @@ -11072,7 +11072,7 @@ public void testInnerJoinQueryOfLookupRemovable(Map queryContext ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.v\", 0, 3)", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.v\", 0, 3)", ColumnType.STRING)) .columns("dim1", "dim2", "v0") .context(queryContext) .build() @@ -11114,7 +11114,7 @@ public void testInnerJoinTwoLookupsToTableUsingNumericColumn(Map expressionVirtualColumn( "v0", "CAST(\"k\", 'DOUBLE')", - ValueType.FLOAT + ColumnType.FLOAT ) ) .columns("k", "v0") @@ -11382,7 +11382,7 @@ public void testWhereInSelectNullFromLookup() throws Exception .setDataSource(new LookupDataSource("lookyloo")) .setInterval(querySegmentSpec(Filtration.eternity())) .setVirtualColumns( - expressionVirtualColumn("v0", "null", ValueType.STRING) + expressionVirtualColumn("v0", "null", ColumnType.STRING) ) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0"))) @@ -11396,7 +11396,7 @@ public void testWhereInSelectNullFromLookup() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "null", ValueType.STRING) + expressionVirtualColumn("v0", "null", ColumnType.STRING) ) .columns("__time", "cnt", "dim2", "dim3", "m1", "m2", "unique_dim1", "v0") .context(QUERY_CONTEXT_DEFAULT) @@ -11522,7 +11522,7 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) throws Exce .dataSource(new LookupDataSource("lookyloo")) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ValueType.FLOAT) + expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ColumnType.FLOAT) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") @@ -11664,7 +11664,7 @@ public void testInnerJoinMismatchedTypes(Map queryContext) throw .dataSource(new LookupDataSource("lookyloo")) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ValueType.FLOAT) + expressionVirtualColumn("v0", "CAST(\"k\", 'DOUBLE')", ColumnType.FLOAT) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") @@ -11742,7 +11742,7 @@ public void testInnerJoinRightFunction(Map queryContext) throws .dataSource(new LookupDataSource("lookyloo")) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "substring(\"k\", 0, 2)", ValueType.STRING) + expressionVirtualColumn("v0", "substring(\"k\", 0, 2)", ColumnType.STRING) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") @@ -12314,10 +12314,10 @@ public void testTimeseriesUsingTimeFloorWithTimeShift() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(timestamp_shift(\"__time\",'P1D',-1,'UTC'),'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec( new DefaultLimitSpec( @@ -12362,10 +12362,10 @@ public void testTimeseriesUsingTimeFloorWithTimestampAdd() throws Exception expressionVirtualColumn( "v0", "timestamp_floor((\"__time\" + -86400000),'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec( new DefaultLimitSpec( @@ -12868,8 +12868,8 @@ public void testGroupByAggregatorDefaultValues() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( @@ -12992,8 +12992,8 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter(selector("dim2", "a", null)) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ValueType.STRING)) - .setDimensions(new DefaultDimensionSpec("v0", "_d0", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( @@ -13143,10 +13143,10 @@ public void testGroupByExtractYear() throws Exception expressionVirtualColumn( "v0", "timestamp_extract(\"__time\",'YEAR','UTC')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec( new DefaultLimitSpec( @@ -13192,10 +13192,10 @@ public void testGroupByFormatYearAndMonth() throws Exception expressionVirtualColumn( "v0", "timestamp_format(\"__time\",'yyyy MM','UTC')", - ValueType.STRING + ColumnType.STRING ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec( new DefaultLimitSpec( @@ -13239,10 +13239,10 @@ public void testGroupByExtractFloorTime() throws Exception expressionVirtualColumn( "v0", "timestamp_extract(timestamp_floor(\"__time\",'P1Y',null,'UTC'),'YEAR','UTC')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -13277,10 +13277,10 @@ public void testGroupByExtractFloorTimeLosAngeles() throws Exception expressionVirtualColumn( "v0", "timestamp_extract(timestamp_floor(\"__time\",'P1Y',null,'America/Los_Angeles'),'YEAR','America/Los_Angeles')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_LOS_ANGELES) .build() @@ -13372,10 +13372,10 @@ public void testTimeseriesWithLimitAndOffset() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec(DefaultLimitSpec.builder().offset(1).limit(2).build()) .setContext(QUERY_CONTEXT_DEFAULT) @@ -13432,13 +13432,13 @@ public void testGroupByTimeAndOtherDimension() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -13497,12 +13497,12 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() throws Exceptio expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("dim2", "d1") ) ) @@ -13524,13 +13524,13 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() throws Exceptio expressionVirtualColumn( "v0", "timestamp_floor(\"d0\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("d1", "_d0"), - new DefaultDimensionSpec("v0", "_d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "_d1", ColumnType.LONG) ) ) .setDimFilter( @@ -13598,18 +13598,18 @@ public void testGroupingSets() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators( @@ -13664,18 +13664,18 @@ public void testGroupingAggregatorDifferentOrder() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators( @@ -13741,7 +13741,7 @@ public void testGroupingAggregatorWithPostAggregator() throws Exception .setGranularity(Granularities.ALL) .setDimensions( dimensions( - new DefaultDimensionSpec("dim2", "d0", ValueType.STRING) + new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING) ) ) .setAggregatorSpecs(aggregators( @@ -13779,7 +13779,7 @@ public void testGroupingSetsWithNumericDimension() throws Exception .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setSubtotalsSpec( ImmutableList.of( @@ -13816,18 +13816,18 @@ public void testGroupByRollup() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -13875,17 +13875,17 @@ public void testGroupByRollupDifferentOrder() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ), expressionVirtualColumn( "v1", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("v1", "d1") ) ) @@ -13932,18 +13932,18 @@ public void testGroupByCube() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -13993,18 +13993,18 @@ public void testGroupingSetsWithDummyDimension() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v2", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v2", "d2", ValueType.LONG) + new DefaultDimensionSpec("v2", "d2", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -14055,18 +14055,18 @@ public void testGroupingSetsNoSuperset() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -14111,18 +14111,18 @@ public void testGroupingSetsWithOrderByDimension() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -14184,18 +14184,18 @@ public void testGroupingSetsWithOrderByAggregator() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -14253,18 +14253,18 @@ public void testGroupingSetsWithOrderByAggregatorWithLimit() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",'')", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0"), - new DefaultDimensionSpec("v1", "d1", ValueType.LONG) + new DefaultDimensionSpec("v1", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -14396,12 +14396,12 @@ public void testUsingSubqueryAsPartOfOrFilter(Map queryContext) .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setVirtualColumns(expressionVirtualColumn("v0", "1", ValueType.LONG)) + .setVirtualColumns(expressionVirtualColumn("v0", "1", ColumnType.LONG)) .setDimFilter(new LikeDimFilter("dim1", "%bc", null, null)) .setDimensions( dimensions( new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setContext(queryContext) @@ -14529,7 +14529,7 @@ public void testNestedGroupByOnInlineDataSourceWithFilter(Map qu new DefaultDimensionSpec("v0", "d0") ) ) - .setVirtualColumns(expressionVirtualColumn("v0", "'def'", ValueType.STRING)) + .setVirtualColumns(expressionVirtualColumn("v0", "'def'", ColumnType.STRING)) .build() ) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -14601,7 +14601,7 @@ public void testGroupByJoinAsNativeQueryWithUnoptimizedFilter(Map queryContext) thr .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimFilter(in("dim1", ImmutableList.of("abc", "def"), null)) - .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(queryContext) .build() @@ -14925,7 +14925,7 @@ public void testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery(Map queryContext) throws E ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) .setContext(QUERY_CONTEXT_DEFAULT) .build() .withOverriddenContext(queryContext) @@ -15193,7 +15193,7 @@ public void testNotInAggregationSubquery(Map queryContext) throw and(selector("_j0.p0", null, null), expressionFilter("(\"j0._a1\" >= \"j0._a0\")")) ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) + .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) .setContext(queryContext) .build() ), @@ -15594,8 +15594,8 @@ public void testSortProjectAfterNestedGroupBy() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions( - new DefaultDimensionSpec("__time", "d0", ValueType.LONG), - new DefaultDimensionSpec("m2", "d1", ValueType.DOUBLE), + new DefaultDimensionSpec("__time", "d0", ColumnType.LONG), + new DefaultDimensionSpec("m2", "d1", ColumnType.DOUBLE), new DefaultDimensionSpec("dim1", "d2") )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -15604,8 +15604,8 @@ public void testSortProjectAfterNestedGroupBy() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions( - new DefaultDimensionSpec("d0", "_d0", ValueType.LONG), - new DefaultDimensionSpec("d2", "_d1", ValueType.STRING) + new DefaultDimensionSpec("d0", "_d0", ColumnType.LONG), + new DefaultDimensionSpec("d2", "_d1", ColumnType.STRING) )) .setAggregatorSpecs( aggregators( @@ -15693,7 +15693,7 @@ public void testPostAggWithTopN() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .dimension(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT)) + .dimension(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT)) .filters("dim2", "a") .aggregators( useDefault @@ -15748,7 +15748,7 @@ public void testConcat() throws Exception .virtualColumns(expressionVirtualColumn( "v0", "concat(\"dim1\",'-',\"dim1\",'_',\"dim1\")", - ValueType.STRING + ColumnType.STRING )) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -15774,7 +15774,7 @@ public void testConcat() throws Exception .virtualColumns(expressionVirtualColumn( "v0", "concat(\"dim1\",concat(\"dim2\",'x'),\"m2\",9999,\"dim1\")", - ValueType.STRING + ColumnType.STRING )) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -15804,7 +15804,7 @@ public void testConcatGroup() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "concat(\"dim1\",'-',\"dim1\",'_',\"dim1\")", - ValueType.STRING + ColumnType.STRING )) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0"))) .setGranularity(Granularities.ALL) @@ -15849,7 +15849,7 @@ public void testConcatGroup() throws Exception .setVirtualColumns(expressionVirtualColumn( "v0", "concat(\"dim1\",concat(\"dim2\",'x'),\"m2\",9999,\"dim1\")", - ValueType.STRING + ColumnType.STRING )) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0"))) .setGranularity(Granularities.ALL) @@ -15870,7 +15870,7 @@ public void testTextcat() throws Exception newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim1\",\"dim1\")", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim1\",\"dim1\")", ColumnType.STRING)) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) @@ -15895,7 +15895,7 @@ public void testTextcat() throws Exception .virtualColumns(expressionVirtualColumn( "v0", "concat(\"dim1\",CAST(\"m2\", 'STRING'))", - ValueType.STRING + ColumnType.STRING )) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -16021,7 +16021,7 @@ public void testRequireTimeConditionPositive() throws Exception new ExtractionDimensionSpec( "dim1", "d0", - ValueType.STRING, + ColumnType.STRING, new SubstringDimExtractionFn(0, 1) ) ) @@ -16237,7 +16237,7 @@ public void testRadiansAndDegrees() throws Exception .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "(toRadians((\"m1\" * 15)) / toDegrees(\"m2\"))", ValueType.DOUBLE) + expressionVirtualColumn("v0", "(toRadians((\"m1\" * 15)) / toDegrees(\"m2\"))", ColumnType.DOUBLE) ) .columns("v0") .filters(selector("dim1", "1", null)) @@ -16271,23 +16271,23 @@ public void testTimestampDiff() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "div((\"__time\" - 915148800000),86400000)", ValueType.LONG), - expressionVirtualColumn("v1", "div((978307200000 - \"__time\"),86400000)", ValueType.LONG), - expressionVirtualColumn("v2", "div((\"__time\" - 946602000000),3600000)", ValueType.LONG), - expressionVirtualColumn("v3", "div((\"__time\" - 946684683000),60000)", ValueType.LONG), - expressionVirtualColumn("v4", "div((\"__time\" - 946684743000),1000)", ValueType.LONG), - expressionVirtualColumn("v5", "subtract_months(\"__time\",941414400000,'UTC')", ValueType.LONG), + expressionVirtualColumn("v0", "div((\"__time\" - 915148800000),86400000)", ColumnType.LONG), + expressionVirtualColumn("v1", "div((978307200000 - \"__time\"),86400000)", ColumnType.LONG), + expressionVirtualColumn("v2", "div((\"__time\" - 946602000000),3600000)", ColumnType.LONG), + expressionVirtualColumn("v3", "div((\"__time\" - 946684683000),60000)", ColumnType.LONG), + expressionVirtualColumn("v4", "div((\"__time\" - 946684743000),1000)", ColumnType.LONG), + expressionVirtualColumn("v5", "subtract_months(\"__time\",941414400000,'UTC')", ColumnType.LONG), expressionVirtualColumn( "v6", "div(subtract_months(\"__time\",846806400000,'UTC'),12)", - ValueType.LONG + ColumnType.LONG ), expressionVirtualColumn( "v7", "div(subtract_months(\"__time\",844128000000,'UTC'),3)", - ValueType.LONG + ColumnType.LONG ), - expressionVirtualColumn("v8", "div(div((\"__time\" - 907200000000),1000),604800)", ValueType.LONG) + expressionVirtualColumn("v8", "div(div((\"__time\" - 907200000000),1000),604800)", ColumnType.LONG) ) .columns("v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7", "v8") .limit(2) @@ -16316,8 +16316,8 @@ public void testTimestampCeil() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns( - expressionVirtualColumn("v0", "946684800000", ValueType.LONG), - expressionVirtualColumn("v1", "946771200000", ValueType.LONG) + expressionVirtualColumn("v0", "946684800000", ColumnType.LONG), + expressionVirtualColumn("v1", "946771200000", ColumnType.LONG) ) .columns("v0", "v1") .limit(1) @@ -16358,10 +16358,10 @@ public void testNvlColumns() throws Exception expressionVirtualColumn( "v0", "case_searched(notnull(\"dim2\"),\"dim2\",\"dim1\")", - ValueType.STRING + ColumnType.STRING ) ) - .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.STRING))) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -16411,7 +16411,7 @@ public void testGroupByWithLiteralInSubqueryGrouping() throws Exception .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim4", "_d0", ValueType.STRING)) + .setDimensions(new DefaultDimensionSpec("dim4", "_d0", ColumnType.STRING)) .setContext(QUERY_CONTEXT_DEFAULT) .build() ) @@ -16419,19 +16419,19 @@ public void testGroupByWithLiteralInSubqueryGrouping() throws Exception expressionVirtualColumn( "v0", "\'dummy\'", - ValueType.STRING + ColumnType.STRING ), expressionVirtualColumn( "v1", "case_searched((\"_d0\" == 'b'),\"_d0\",null)", - ValueType.STRING + ColumnType.STRING ) ) .setInterval(querySegmentSpec(Filtration.eternity())) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.STRING), - new DefaultDimensionSpec("v1", "d1", ValueType.STRING) + new DefaultDimensionSpec("v0", "d0", ColumnType.STRING), + new DefaultDimensionSpec("v1", "d1", ColumnType.STRING) ) ) .setGranularity(Granularities.ALL) @@ -16599,7 +16599,7 @@ public void testCountOnSemiJoinSingleColumn(Map queryContext) th ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns("v0") .context(queryContext) .build() @@ -16637,7 +16637,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ) ) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -16655,7 +16655,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ) ) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -16667,7 +16667,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "_v0") .filters(new SelectorDimFilter("v0", "10.1", null)) .context(queryContext) @@ -16708,7 +16708,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAcces ) ) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -16726,7 +16726,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAcces JodaUtils.MAX_INSTANT ) )) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "v0") .context(queryContext) .build() @@ -16757,7 +16757,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -16779,7 +16779,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "_v0") .filters(new SelectorDimFilter("v0", "10.1", null)) .context(queryContext) @@ -16825,7 +16825,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAcces ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "v0") .context(queryContext) .build() @@ -16856,7 +16856,7 @@ public void testLeftJoinOnTwoInlineDataSources(Map queryContext) .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -16878,7 +16878,7 @@ public void testLeftJoinOnTwoInlineDataSources(Map queryContext) ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "_v0") .context(queryContext) .build() @@ -16923,7 +16923,7 @@ public void testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess(Map queryContext .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(new SelectorDimFilter("dim1", "10.1", null)) - .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -17075,7 +17075,7 @@ public void testInnerJoinOnTwoInlineDataSources(Map queryContext ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING)) .columns("__time", "_v0") .context(queryContext) .build() @@ -17120,7 +17120,7 @@ public void testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess(Map ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .dimension(new DefaultDimensionSpec("j0.dim4", "_d0", ValueType.STRING)) + .dimension(new DefaultDimensionSpec("j0.dim4", "_d0", ColumnType.STRING)) .threshold(4) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(queryContext) @@ -17283,7 +17283,7 @@ public void testTopNOnStringWithNonSortedOrUniqueDictionaryOrderByDim(Map queryConte .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "concat(\"k\",'')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"k\",'')", ColumnType.STRING) ) .setDimensions(new DefaultDimensionSpec("v0", "d0")) .build() @@ -18168,7 +18168,7 @@ public void testLeftJoinSubqueryWithNullKeyFilter(Map queryConte .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "concat(\"k\",'')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"k\",'')", ColumnType.STRING) ) .setDimensions(new DefaultDimensionSpec("v0", "d0")) .build() @@ -18230,7 +18230,7 @@ public void testLeftJoinSubqueryWithSelectorFilter(Map queryCont .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "concat(\"k\",'')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"k\",'')", ColumnType.STRING) ) .setDimensions(new DefaultDimensionSpec("v0", "d0")) .build() @@ -18320,7 +18320,7 @@ public void testInnerJoinSubqueryWithSelectorFilter(Map queryCon .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setVirtualColumns( - expressionVirtualColumn("v0", "concat(\"k\",'')", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"k\",'')", ColumnType.STRING) ) .setDimensions(new DefaultDimensionSpec("v0", "d0")) .build() @@ -18640,7 +18640,7 @@ public void testStringAggExpression() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns( - expressionVirtualColumn("v0", "concat(\"dim1\",\"dim2\")", ValueType.STRING) + expressionVirtualColumn("v0", "concat(\"dim1\",\"dim2\")", ColumnType.STRING) ) .aggregators( aggregators( @@ -18786,14 +18786,14 @@ public void testHumanReadableFormatFunction() throws Exception // NOTE: the first expression HUMAN_READABLE_BINARY_BYTE_FORMAT(45678) in SQL is calculated during SQL parse phase, // so the converted Druid native query is its result intead of the raw function call // - .virtualColumns(expressionVirtualColumn("v0", "'44.61 KiB'", ValueType.STRING), - expressionVirtualColumn("v1", "human_readable_binary_byte_format((\"m1\" * 12345))", ValueType.STRING), - expressionVirtualColumn("v2", "human_readable_binary_byte_format((\"m1\" * 12345),0)", ValueType.STRING), - expressionVirtualColumn("v3", "human_readable_decimal_byte_format((\"m1\" * 12345))", ValueType.STRING), - expressionVirtualColumn("v4", "human_readable_decimal_format((\"m1\" * 12345))", ValueType.STRING), - expressionVirtualColumn("v5", "human_readable_binary_byte_format(\"l1\")", ValueType.STRING), - expressionVirtualColumn("v6", "human_readable_decimal_byte_format(\"l1\")", ValueType.STRING), - expressionVirtualColumn("v7", "human_readable_decimal_format(\"l1\")", ValueType.STRING) + .virtualColumns(expressionVirtualColumn("v0", "'44.61 KiB'", ColumnType.STRING), + expressionVirtualColumn("v1", "human_readable_binary_byte_format((\"m1\" * 12345))", ColumnType.STRING), + expressionVirtualColumn("v2", "human_readable_binary_byte_format((\"m1\" * 12345),0)", ColumnType.STRING), + expressionVirtualColumn("v3", "human_readable_decimal_byte_format((\"m1\" * 12345))", ColumnType.STRING), + expressionVirtualColumn("v4", "human_readable_decimal_format((\"m1\" * 12345))", ColumnType.STRING), + expressionVirtualColumn("v5", "human_readable_binary_byte_format(\"l1\")", ColumnType.STRING), + expressionVirtualColumn("v6", "human_readable_decimal_byte_format(\"l1\")", ColumnType.STRING), + expressionVirtualColumn("v7", "human_readable_decimal_format(\"l1\")", ColumnType.STRING) ) .columns("m1", "v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7") .filters(selector("dim1", "1", null)) @@ -18884,7 +18884,7 @@ public void testCommonVirtualExpressionWithDifferentValueType() throws Exception expressionVirtualColumn( "v0", "'none'", - ValueType.STRING + ColumnType.STRING ) ) .dimension( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSimpleQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSimpleQueryTest.java index 0ac759caaeb4..f749e1535c71 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSimpleQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSimpleQueryTest.java @@ -31,7 +31,7 @@ import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction; import org.apache.druid.query.ordering.StringComparators; -import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.util.CalciteTests; import org.junit.Test; @@ -57,12 +57,12 @@ public void testGroupByTimeAndDim() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("dim2", "d1") ) ) @@ -105,13 +105,13 @@ public void testGroupByDimAndTime() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -154,13 +154,13 @@ public void testGroupByDimAndTimeWhereOnTime() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -198,13 +198,13 @@ public void testGroupByDimAndTimeOnDim() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setDimFilter(new LikeDimFilter("dim2", "a%", null, null)) @@ -243,12 +243,12 @@ public void testGroupByTimeAndDimOrderByDim() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("dim2", "d1") ) ) @@ -298,12 +298,12 @@ public void testGroupByTimeAndDimOrderByDimDesc() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( - new DefaultDimensionSpec("v0", "d0", ValueType.LONG), + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), new DefaultDimensionSpec("dim2", "d1") ) ) @@ -353,13 +353,13 @@ public void testGroupByDimAndTimeOrderByTime() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -410,13 +410,13 @@ public void testGroupByDimAndTimeOrderByTimeDesc() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -467,13 +467,13 @@ public void testGroupByDimAndTimeOrderByTimeAndDim() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -525,13 +525,13 @@ public void testGroupByDimAndTimeOrderByDimAndTime() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG) + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -583,13 +583,13 @@ public void testGroupByDimAndTimeAndDimOrderByDimAndTimeDim() throws Exception expressionVirtualColumn( "v0", "timestamp_floor(\"__time\",'P1M',null,'UTC')", - ValueType.LONG + ColumnType.LONG ) ) .setDimensions( dimensions( new DefaultDimensionSpec("dim2", "d0"), - new DefaultDimensionSpec("v0", "d1", ValueType.LONG), + new DefaultDimensionSpec("v0", "d1", ColumnType.LONG), new DefaultDimensionSpec("dim1", "d2") ) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index 6124473d3272..862793279b07 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -37,8 +37,8 @@ import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.query.filter.SearchQueryDimFilter; import org.apache.druid.query.search.ContainsSearchQuerySpec; +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.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.expression.builtin.ContainsOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion; @@ -74,26 +74,26 @@ public class ExpressionsTest extends ExpressionTestBase { private static final RowSignature ROW_SIGNATURE = RowSignature .builder() - .add("t", ValueType.LONG) - .add("a", ValueType.LONG) - .add("b", ValueType.LONG) - .add("p", ValueType.LONG) - .add("x", ValueType.FLOAT) - .add("y", ValueType.LONG) - .add("z", ValueType.FLOAT) - .add("s", ValueType.STRING) - .add("nan", ValueType.DOUBLE) - .add("inf", ValueType.DOUBLE) - .add("-inf", ValueType.DOUBLE) - .add("fnan", ValueType.FLOAT) - .add("finf", ValueType.FLOAT) - .add("-finf", ValueType.FLOAT) - .add("hexstr", ValueType.STRING) - .add("intstr", ValueType.STRING) - .add("spacey", ValueType.STRING) - .add("newliney", ValueType.STRING) - .add("tstr", ValueType.STRING) - .add("dstr", ValueType.STRING) + .add("t", ColumnType.LONG) + .add("a", ColumnType.LONG) + .add("b", ColumnType.LONG) + .add("p", ColumnType.LONG) + .add("x", ColumnType.FLOAT) + .add("y", ColumnType.LONG) + .add("z", ColumnType.FLOAT) + .add("s", ColumnType.STRING) + .add("nan", ColumnType.DOUBLE) + .add("inf", ColumnType.DOUBLE) + .add("-inf", ColumnType.DOUBLE) + .add("fnan", ColumnType.FLOAT) + .add("finf", ColumnType.FLOAT) + .add("-finf", ColumnType.FLOAT) + .add("hexstr", ColumnType.STRING) + .add("intstr", ColumnType.STRING) + .add("spacey", ColumnType.STRING) + .add("newliney", ColumnType.STRING) + .add("tstr", ColumnType.STRING) + .add("dstr", ColumnType.STRING) .build(); private static final Map BINDINGS = ImmutableMap.builder() @@ -521,7 +521,7 @@ public void testRegexpLikeAsFilter() new ExpressionVirtualColumn( "v0", "concat('Z',\"s\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ), @@ -1242,7 +1242,7 @@ public void testContainsAsFilter() new ExpressionVirtualColumn( "v0", "concat('what is',\"spacey\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ), @@ -1264,7 +1264,7 @@ public void testContainsAsFilter() new ExpressionVirtualColumn( "v0", "concat('what is',\"spacey\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ), @@ -1286,7 +1286,7 @@ public void testContainsAsFilter() new ExpressionVirtualColumn( "v0", "concat('what is',\"spacey\")", - ValueType.STRING, + ColumnType.STRING, TestExprMacroTable.INSTANCE ) ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java index 58278232ddd4..2020ab0f284f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java @@ -25,8 +25,8 @@ import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.druid.java.util.common.DateTimes; +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.sql.calcite.expression.builtin.GreatestOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -47,9 +47,9 @@ public class GreatestExpressionTest extends ExpressionTestBase private static final String STRING_VALUE = "foo"; private static final RowSignature ROW_SIGNATURE = RowSignature .builder() - .add(DOUBLE_KEY, ValueType.DOUBLE) - .add(LONG_KEY, ValueType.LONG) - .add(STRING_KEY, ValueType.STRING) + .add(DOUBLE_KEY, ColumnType.DOUBLE) + .add(LONG_KEY, ColumnType.LONG) + .add(STRING_KEY, ColumnType.STRING) .build(); private static final Map BINDINGS = ImmutableMap.of( DOUBLE_KEY, DOUBLE_VALUE, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java index 9318028d917f..4cc354766a89 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.rex.RexNode; +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.sql.calcite.expression.builtin.IPv4AddressMatchOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -48,7 +48,7 @@ public class IPv4AddressMatchExpressionTest extends ExpressionTestBase private static final long NO_MATCH = 0L; private static final String VAR = "s"; - private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.STRING).build(); + private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ColumnType.STRING).build(); private static final Map BINDINGS = ImmutableMap.of(VAR, "foo"); private IPv4AddressMatchOperatorConversion target; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java index 3d8956c52e72..c9616b2eba65 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java @@ -22,8 +22,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.rex.RexNode; import org.apache.druid.common.config.NullHandling; +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.sql.calcite.expression.builtin.IPv4AddressParseOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -41,7 +41,7 @@ public class IPv4AddressParseExpressionTest extends ExpressionTestBase private static final Long NULL = NullHandling.replaceWithDefault() ? NullHandling.ZERO_LONG : null; private static final String VAR = "f"; - private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.FLOAT).build(); + private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ColumnType.FLOAT).build(); private static final Map BINDINGS = ImmutableMap.of(VAR, 3.14); private IPv4AddressParseOperatorConversion target; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java index 5a0a50e7d851..94ca130e399c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java @@ -21,8 +21,8 @@ import com.google.common.collect.ImmutableMap; import org.apache.calcite.rex.RexNode; +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.sql.calcite.expression.builtin.IPv4AddressStringifyOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -40,7 +40,7 @@ public class IPv4AddressStringifyExpressionTest extends ExpressionTestBase private static final String NULL = null; private static final String VAR = "f"; - private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ValueType.FLOAT).build(); + private static final RowSignature ROW_SIGNATURE = RowSignature.builder().add(VAR, ColumnType.FLOAT).build(); private static final Map BINDINGS = ImmutableMap.of(VAR, 3.14); private IPv4AddressStringifyOperatorConversion target; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java index 6eed91bb241c..b62b5e3e2b46 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java @@ -25,8 +25,8 @@ import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.druid.java.util.common.DateTimes; +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.sql.calcite.expression.builtin.LeastOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -47,9 +47,9 @@ public class LeastExpressionTest extends ExpressionTestBase private static final String STRING_VALUE = "foo"; private static final RowSignature ROW_SIGNATURE = RowSignature .builder() - .add(DOUBLE_KEY, ValueType.DOUBLE) - .add(LONG_KEY, ValueType.LONG) - .add(STRING_KEY, ValueType.STRING) + .add(DOUBLE_KEY, ColumnType.DOUBLE) + .add(LONG_KEY, ColumnType.LONG) + .add(STRING_KEY, ColumnType.STRING) .build(); private static final Map BINDINGS = ImmutableMap.of( DOUBLE_KEY, DOUBLE_VALUE, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/TimeFormatOperatorConversionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/TimeFormatOperatorConversionTest.java index 36a8bb05ace0..fffe75f3c747 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/TimeFormatOperatorConversionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/TimeFormatOperatorConversionTest.java @@ -24,8 +24,8 @@ import org.apache.calcite.rex.RexNode; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; +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.sql.calcite.expression.builtin.TimeFormatOperatorConversion; import org.junit.Before; import org.junit.Test; @@ -40,7 +40,7 @@ public class TimeFormatOperatorConversionTest extends ExpressionTestBase { private static final RowSignature ROW_SIGNATURE = RowSignature .builder() - .add("t", ValueType.LONG) + .add("t", ColumnType.LONG) .build(); private static final Map BINDINGS = ImmutableMap .builder() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/filtration/FiltrationTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/filtration/FiltrationTest.java index 7bfae93fcc54..901ab17a712f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/filtration/FiltrationTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/filtration/FiltrationTest.java @@ -24,8 +24,8 @@ import org.apache.druid.query.filter.IntervalDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.column.ColumnHolder; +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.sql.calcite.util.CalciteTestBase; import org.junit.Assert; import org.junit.Test; @@ -44,7 +44,7 @@ public void testNotIntervals() ) ), null - ).optimize(RowSignature.builder().add(ColumnHolder.TIME_COLUMN_NAME, ValueType.LONG).build()); + ).optimize(RowSignature.builder().add(ColumnHolder.TIME_COLUMN_NAME, ColumnType.LONG).build()); Assert.assertEquals( ImmutableList.of(Filtration.eternity()), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRuleTest.java index 3cf88b9118ff..b9dab37e9bbf 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRuleTest.java @@ -23,8 +23,8 @@ import org.apache.calcite.rel.core.Project; import org.apache.calcite.util.mapping.Mappings; import org.apache.druid.query.TableDataSource; +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.sql.calcite.rel.DruidJoinQueryRel; import org.apache.druid.sql.calcite.rel.DruidOuterQueryRel; import org.apache.druid.sql.calcite.rel.DruidQueryRel; @@ -47,8 +47,8 @@ public class DruidUnionDataSourceRuleTest new TableDataSource("foo"), RowSignature.builder() .addTimeColumn() - .add("col1", ValueType.STRING) - .add("col2", ValueType.LONG) + .add("col1", ColumnType.STRING) + .add("col2", ColumnType.LONG) .build(), false, false diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index d25a9026f150..7522f2dbecf7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -68,8 +68,8 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; 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.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.segment.loading.SegmentLoader; @@ -1392,13 +1392,13 @@ private static void verifyTypes(final List rows, final RowSignature si for (int i = 0; i < row.length; i++) { final Class expectedClass; - final ValueType columnType = + final ColumnType columnType = signature.getColumnType(i) .orElseThrow(() -> new ISE("Encountered null column type")); final boolean nullable = rowType.getFieldList().get(i).getType().isNullable(); - switch (columnType) { + switch (columnType.getType()) { case LONG: expectedClass = Long.class; break; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 79fc67310ccb..b89d017f44c9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -82,8 +82,8 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; 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.incremental.IncrementalIndexSchema; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.Joinable; @@ -704,17 +704,17 @@ public AuthenticationResult createEscalatedAuthenticationResult() x.get("l2") }).collect(Collectors.toList()), RowSignature.builder() - .add("dim1", ValueType.STRING) - .add("dim2", ValueType.STRING) - .add("dim3", ValueType.STRING) - .add("dim4", ValueType.STRING) - .add("dim5", ValueType.STRING) - .add("d1", ValueType.DOUBLE) - .add("d2", ValueType.DOUBLE) - .add("f1", ValueType.FLOAT) - .add("f2", ValueType.FLOAT) - .add("l1", ValueType.LONG) - .add("l2", ValueType.LONG) + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .add("dim3", ColumnType.STRING) + .add("dim4", ColumnType.STRING) + .add("dim5", ColumnType.STRING) + .add("d1", ColumnType.DOUBLE) + .add("d2", ColumnType.DOUBLE) + .add("f1", ColumnType.FLOAT) + .add("f2", ColumnType.FLOAT) + .add("l1", ColumnType.LONG) + .add("l2", ColumnType.LONG) .build() );